Faust - Python Stream Processing¶
# Python Streams ٩(◕‿◕)۶
# Forever scalable event processing & in-memory durable K/V store;
# w/ asyncio & static typing.
import faust
Faust is a stream processing library, porting the ideas from Kafka Streams to Python.
It is used at Robinhood to build high performance distributed systems and real-time data pipelines that process billions of events every day.
Faust provides both stream processing and event processing, sharing similarity with tools such as Kafka Streams, Apache Spark/Storm/Samza/Flink,
It does not use a DSL, it’s just Python! This means you can use all your favorite Python libraries when stream processing: NumPy, PyTorch, Pandas, NLTK, Django, Flask, SQLAlchemy, ++
Faust requires Python 3.6 or later for the new async/await syntax, and variable type annotations.
Here’s an example processing a stream of incoming orders:
app = faust.App('myapp', broker='kafka://localhost')
# Models describe how messages are serialized:
# {"account_id": "3fae-...", amount": 3}
class Order(faust.Record):
account_id: str
amount: int
@app.agent(value_type=Order)
async def order(orders):
async for order in orders:
# process infinite stream of orders.
print(f'Order for {order.account_id}: {order.amount}')
The Agent decorator defines a “stream processor” that essentially consumes from a Kafka topic and does something for every event it receives.
The agent is an async def
function, so can also perform
other operations asynchronously, such as web requests.
This system can persist state, acting like a database. Tables are named distributed key/value stores you can use as regular Python dictionaries.
Tables are stored locally on each machine using a super fast embedded database written in C++, called RocksDB.
Tables can also store aggregate counts that are optionally “windowed” so you can keep track of “number of clicks from the last day,” or “number of clicks in the last hour.” for example. Like Kafka Streams, we support tumbling, hopping and sliding windows of time, and old windows can be expired to stop data from filling up.
For reliability we use a Kafka topic as “write-ahead-log”. Whenever a key is changed we publish to the changelog. Standby nodes consume from this changelog to keep an exact replica of the data and enables instant recovery should any of the nodes fail.
To the user a table is just a dictionary, but data is persisted between restarts and replicated across nodes so on failover other nodes can take over automatically.
You can count page views by URL:
# data sent to 'clicks' topic sharded by URL key.
# e.g. key="http://example.com" value="1"
click_topic = app.topic('clicks', key_type=str, value_type=int)
# default value for missing URL will be 0 with `default=int`
counts = app.Table('click_counts', default=int)
@app.agent(click_topic)
async def count_click(clicks):
async for url, count in clicks.items():
counts[url] += count
The data sent to the Kafka topic is partitioned, which means the clicks will be sharded by URL in such a way that every count for the same URL will be delivered to the same Faust worker instance.
Faust supports any type of stream data: bytes, Unicode and serialized structures, but also comes with “Models” that use modern Python syntax to describe how keys and values in streams are serialized:
# Order is a json serialized dictionary,
# having these fields:
class Order(faust.Record):
account_id: str
product_id: str
price: float
quantity: float = 1.0
orders_topic = app.topic('orders', key_type=str, value_type=Order)
@app.agent(orders_topic)
async def process_order(orders):
async for order in orders:
# process each order using regular Python
total_price = order.price * order.quantity
await send_order_received_email(order.account_id, order)
Faust is statically typed, using the mypy type checker, so you can take advantage of static types when writing applications.
The Faust source code is small, well organized, and serves as a good resource for learning the implementation of Kafka Streams.
- Learn more about Faust in the Introducing Faust introduction page
to read more about Faust, system requirements, installation instructions, community resources, and more.
- or go directly to the Quick Start tutorial
to see Faust in action by programming a streaming application.
- then explore the User Guide
for in-depth information organized by topic.
Contents¶
Copyright¶
Faust User Manual
Copyright © 2017-2019, Robinhood Markets, Inc.
All rights reserved. This material may be copied or distributed only subject to the terms and conditions set forth in the Creative Commons Attribution-ShareAlike 4.0 International <http://creativecommons.org/licenses/by-sa/4.0/legalcode>`_ license.
You may share and adapt the material, even for commercial purposes, but you must give the original author credit. If you alter, transform, or build upon this work, you may distribute the resulting work only under the same license or a license compatible to this one.
Note
While the Faust documentation is offered under the Creative Commons Attribution-ShareAlike 4.0 International license the Faust software is offered under the BSD License (3 Clause)
Introducing Faust¶
- Version
1.5.5
- Web
- Download
- Source
- Keywords
distributed, stream, async, processing, data, queue
Table of Contents
What can it do?¶
- Agents
Process infinite streams in a straightforward manner using asynchronous generators. The concept of “agents” comes from the actor model, and means the stream processor can execute concurrently on many CPU cores, and on hundreds of machines at the same time.
Use regular Python syntax to process streams and reuse your favorite libraries:
@app.agent() async def process(stream): async for value in stream: process(value)
- Tables
Tables are sharded dictionaries that enable stream processors to be stateful with persistent and durable data.
Streams are partitioned to keep relevant data close, and can be easily repartitioned to achieve the topology you need.
In this example we repartition an order stream by account id, to count orders in a distributed table:
import faust # this model describes how message values are serialized # in the Kafka "orders" topic. class Order(faust.Record, serializer='json'): account_id: str product_id: str amount: int price: float orders_kafka_topic = app.topic('orders', value_type=Order) # our table is sharded amongst worker instances, and replicated # with standby copies to take over if one of the nodes fail. order_count_by_account = app.Table('order_count', default=int) @app.agent(orders_kafka_topic) async def process(orders: faust.Stream[Order]) -> None: async for order in orders.group_by(Order.account_id): order_count_by_account[order.account_id] += 1
If we start multiple instances of this Faust application on many machines, any order with the same account id will be received by the same stream processing agent, so the count updates correctly in the table.
Sharding/partitioning is an essential part of stateful stream processing applications, so take this into account when designing your system, but note that streams can also be processed in round-robin order so you can use Faust for event processing and as a task queue also.
- Asynchronous with
asyncio
Faust takes full advantage of
asyncio
and the newasync
/await
keywords in Python 3.6+ to run multiple stream processors in the same process, along with web servers and other network services.Thanks to Faust and
asyncio
you can now embed your stream processing topology into your existingasyncio
/gevent/ eventlet/Twisted/Tornado applications.- Faust is…
- Simple
Faust is extremely easy to use. To get started using other stream processing solutions you have complicated hello-world projects, and infrastructure requirements. Faust only requires Kafka, the rest is just Python, so If you know Python you can already use Faust to do stream processing, and it can integrate with just about anything.
Here’s one of the easier applications you can make:
import faust class Greeting(faust.Record): from_name: str to_name: str app = faust.App('hello-app', broker='kafka://localhost') topic = app.topic('hello-topic', value_type=Greeting) @app.agent(topic) async def hello(greetings): async for greeting in greetings: print(f'Hello from {greeting.from_name} to {greeting.to_name}') @app.timer(interval=1.0) async def example_sender(app): await hello.send( value=Greeting(from_name='Faust', to_name='you'), ) if __name__ == '__main__': app.main()
You’re probably a bit intimidated by the async and await keywords, but you don’t have to know how
asyncio
works to use Faust: just mimic the examples, and you’ll be fine.The example application starts two tasks: one is processing a stream, the other is a background thread sending events to that stream. In a real-life application, your system will publish events to Kafka topics that your processors can consume from, and the background thread is only needed to feed data into our example.
- Highly Available
Faust is highly available and can survive network problems and server crashes. In the case of node failure, it can automatically recover, and tables have standby nodes that will take over.
- Distributed
Start more instances of your application as needed.
- Fast
A single-core Faust worker instance can already process tens of thousands of events every second, and we are reasonably confident that throughput will increase once we can support a more optimized Kafka client.
- Flexible
Faust is just Python, and a stream is an infinite asynchronous iterator. If you know how to use Python, you already know how to use Faust, and it works with your favorite Python libraries like Django, Flask, SQLAlchemy, NTLK, NumPy, SciPy, TensorFlow, etc.
Faust is used for…
|
|
How do I use it?¶
Step 1: Add events to your system
Was an account created? Publish to Kafka.
Did a user change their password? Publish to Kafka.
Did someone make an order, create a comment, tag something, …? Publish it all to Kafka!
Step 2: Use Faust to process those events
Some ideas based on the events mentioned above:
Send email when an order is dispatches.
Find orders created with no corresponding dispatch event for more than three consecutive days.
Find accounts that changed their password from a suspicious IP address.
Starting to get the idea?
What do I need?¶
Faust requires Python 3.6 or later, and a running Kafka broker.
There’s no plan to support earlier Python versions. Please get in touch if this is something you want to work on.
Extensions¶
Name |
Version |
Bundle |
|
5.0 |
|
|
aredis 1.1 |
|
|
0.20.0 |
|
|
3.2.1 |
|
|
0.8.1 |
|
|
1.4.0 |
|
|
1.16.0 |
|
Optimizations¶
These can be all installed using pip install faust[fast]
:
Name |
Version |
Bundle |
|
1.1.0 |
|
|
1.1.0 |
|
|
2.1.0 |
|
|
0.9.26 |
|
|
1.1.0 |
|
Debugging extras¶
These can be all installed using pip install faust[debug]
:
Name |
Version |
Bundle |
|
0.3 |
|
|
1.1.0 |
|
Note
See bundles in the Installation instructions section of this document for a list of supported setuptools extensions.
To specify multiple extensions at the same time
separate extensions with the comma:
$ pip install faust[uvloop,fast,rocksdb,datadog,redis]
RocksDB On MacOS Sierra
To install python-rocksdb on MacOS Sierra you need to specify some additional compiler flags:
$ CFLAGS='-std=c++11 -stdlib=libc++ -mmacosx-version-min=10.10' \
pip install -U --no-cache python-rocksdb
Design considerations¶
- Modern Python
Faust uses current Python 3 features such as
async
/await
and type annotations. It’s statically typed and verified by the mypy type checker. You can take advantage of type annotations when writing Faust applications, but this is not mandatory.- Library
Faust is designed to be used as a library, and embeds into any existing Python program, while also including helpers that make it easy to deploy applications without boilerplate.
- Supervised
The Faust worker is built up by many different services that start and stop in a certain order. These services can be managed by supervisors, but if encountering an irrecoverable error such as not recovering from a lost Kafka connections, Faust is designed to crash.
For this reason Faust is designed to run inside a process supervisor tool such as supervisord, Circus, or one provided by your Operating System.
- Extensible
Faust abstracts away storages, serializers, and even message transports, to make it easy for developers to extend Faust with new capabilities, and integrate into your existing systems.
- Lean
The source code is short and readable and serves as a good starting point for anyone who wants to learn how Kafka stream processing systems work.
Getting Help¶
Mailing list¶
For discussions about the usage, development, and future of Faust, please join the faust-users mailing list.
Resources¶
Bug tracker¶
If you have any suggestions, bug reports, or annoyances please report them to our issue tracker at https://github.com/robinhood/faust/issues/
Playbooks¶
- Release
1.5
- Date
Apr 17, 2019
Quick Start¶
Hello World¶
Application¶
The first thing you need to get up and running with Faust is to define an application.
The application (or app for short) configures your project and implements common functionality. We also define a topic description, and an agent to process messages in that topic.
Lets create the file hello_world.py:
import faust
app = faust.App(
'hello-world',
broker='kafka://localhost:9092',
value_serializer='raw',
)
greetings_topic = app.topic('greetings')
@app.agent(greetings_topic)
async def greet(greetings):
async for greeting in greetings:
print(greeting)
In this tutorial, we keep everything in a single module, but for larger projects, you can create a dedicated package with a submodule layout.
The first argument passed to the app is the id
of the application, needed
for internal bookkeeping and to distribute work among worker instances.
By default Faust will use JSON serialization, so we specify value_serializer
here as raw
to avoid deserializing incoming greetings. For real
applications you should define models (see Models, Serialization, and Codecs).
Here you defined a Kafka topic greetings
and then iterated over the
messages in the topic and printed each one of them.
Note
The application id
setting (i.e. 'hello-world'
in
the example above), should be unique per Faust app in your Kafka
cluster.
Starting Kafka¶
Before running your app, you need to start Zookeeper and Kafka.
Start Zookeeper first:
$ $KAFKA_HOME/bin/zookeeper-server-start $KAFKA_HOME/etc/kafka/zookeeper.properties
Then start Kafka:
$ $KAFKA_HOME/bin/kafka-server-start $KAFKA_HOME/etc/kafka/server.properties
Running the Faust worker¶
Now that you have created a simple Faust application and have Kafka and Zookeeper running, you need to run a worker instance for the application.
Start a worker:
$ faust -A hello_world worker -l info
Multiple instances of a Faust worker can be started independently to distribute stream processing across machines and CPU cores.
In production, you’ll want to run the worker in the background as a daemon. Use the tools provided by your platform, or use something like supervisord.
Use --help
to get a complete listing of available command-line options:
$ faust worker --help
Seeing things in Action¶
At this point, you have an application running, but not much is happening. You need to feed data into the Kafka topic to see Faust print the greetings as it processes the stream, and right now that topic is probably empty.
Let’s use the faust send command to push some messages into the
greetings
topic:
$ faust -A hello_world send @greet "Hello Faust"
The above command sends a message to the greet
agent by using the @
prefix. If you don’t use the prefix, it will be treated as the name of a topic:
$ faust -A hello_world send greetings "Hello Kafka topic"
After sending the messages, you can see your worker start processing them and print the greetings to the console.
Where to go from here…¶
Now that you have seen a simple Faust application in action, you should dive into the other sections of the User Guide or jump right into the Playbooks for tutorials and solutions to common patterns.
Tutorial: Count page views¶
In the Quick Start tutorial, we went over a simple example reading through a stream of greetings and printing them to the console. In this playbook we do something more meaningful with an incoming stream, we’ll maintain real-time counts of page views from a stream of page views.
Application¶
As we did in the Quick Start tutorial, we first define our application.
Let’s create the module page_views.py
:
import faust
app = faust.App(
'page_views',
broker='kafka://localhost:9092',
topic_partitions=4,
)
The topic_partitions
setting defines the maximum number
of workers we can distribute the workload to (also sometimes referred as
the “sharding factor”). In this example, we set this to 4, but in a
production app, we ideally use a higher number.
Page View¶
Let’s now define a model that each page view event from the stream deserializes into. The record is used for JSON dictionaries and describes fields much like the new dataclasses in Python 3.7:
Create a model for our page view event:
class PageView(faust.Record):
id: str
user: str
Type annotations are used not only for defining static types, but also to define how fields are deserialized, and lets you specify models that contains other models, and so on. See the Models, Serialization, and Codecs guide for more information.
Input Stream¶
Next we define the source topic to read the “page view” events from, and we specify that every value in this topic is of the PageView type.
page_view_topic = app.topic('page_views', value_type=PageView)
Counts Table¶
Then we define a Table. This is like a Python dictionary, but is distributed across the cluster, partitioned by the dictionary key.
page_views = app.Table('page_views', default=int)
Count Page Views¶
Now that we have defined our input stream, as well as a table to maintain counts, we define an agent reading each page view event coming into the stream, always incrementing the count for that page in the table.
Create the agent:
@app.agent(page_view_topic)
async def count_page_views(views):
async for view in views.group_by(PageView.id):
page_views[view.id] += 1
Note
Here we use group_by
to repartition the input stream by
the page id. This is so that we maintain counts on each instance sharded
by the page id. This way in the case of failure, when we move the
processing of some partition to another node, the counts for that
partition (hence, those page ids) also move together.
Now that we written our project, let’s try running it to see the counts update in the changelog topic for the table.
Starting Kafka¶
Before starting a worker, you need to start Zookeeper and Kafka.
First start Zookeeper:
$ $KAFKA_HOME/bin/zookeeper-server-start $KAFKA_HOME/etc/kafka/zookeeper.properties
Then start Kafka:
$ $KAFKA_HOME/bin/kafka-server-start $KAFKA_HOME/etc/kafka/server.properties
Starting the Faust worker¶
Start the worker, similar to what we did in the Quick Start tutorial:
$ faust -A page_views worker -l info
Seeing it in action¶
Now let’s produce some fake page views to see things in action. Send
this data to the page_views
topic:
$ faust -A page_views send page_views '{"id": "foo", "user": "bar"}'
Look at the changelog topic to see the counts. To look at the changelog topic we will use the Kafka console consumer.
$ $KAFKA_HOME/bin/kafka-console-consumer --topic page_views-page_views-changelog --bootstrap-server localhost:9092 --property print.key=True --from-beginning
Note
By default the changelog topic for a given Table
has the format
<app_id>-<table_name>-changelog
Tutorial: Leader Election¶
Faust processes streams of data forming pipelines. Sometimes steps in the pipeline require synchronization, but instead of using mutexes, a better solution is to have one of the workers elected as the leader.
An example of such an application is a news crawler. We can elect one of the workers to be the leader, and the leader maintains all subscriptions (the sources to crawl), then periodically tells the other workers in the cluster to process them.
To demonstrate this we implement a straightforward example where we elect one of our workers as the leader. This leader then periodically send out random greetings to be printed out by available workers.
Application¶
As we did in the Tutorial: Count page views tutorial, we first define your application.
Create a module named leader.py
:
# examples/leader.py
import faust
app = faust.App(
'leader-example',
broker='kafka://localhost:9092',
value_serializer='raw',
)
Greetings Agent¶
Next we define the say
agent
that will get greetings from the
leader and print them out to the console.
Create the agent:
@app.agent()
async def say(greetings):
async for greeting in greetings:
print(greeting)
See also
The guide-agents guide – for more information about agents.
Leader Timer¶
Now define a timer
with the on_leader
flag enabled
so it only executes on the leader.
The timer
will periodically send out a random greeting, to be printed
by one of the workers in the cluster.
Create the leader timer:
import random
@app.timer(2.0, on_leader=True)
async def publish_greetings():
print('PUBLISHING ON LEADER!')
greeting = str(random.random())
await say.send(value=greeting)
Note
The greeting could be picked up by the agent say
on any one of the
running instances.
Starting Kafka¶
To run the project you first need to start Zookeeper and Kafka.
Start Zookeeper:
$ $KAFKA_HOME/bin/zookeeper-server-start $KAFKA_HOME/etc/kafka/zookeeper.properties
Then start Kafka:
$ $KAFKA_HOME/bin/kafka-server-start $KAFKA_HOME/etc/kafka/server.properties
Starting the Faust worker¶
Start the Faust worker, similarly to how we do it in the Quick Start tutorial:
$ faust -A leader worker -l info --web-port 6066
Let’s start two more workers in different terminals on the same machine:
$ faust -A leader worker -l info --web-port 6067
$ faust -A leader worker -l info --web-port 6068
Seeing things in Action¶
Next try to arbitrary shut down (Control-c) some of the workers, to see how the leader stays at just one worker - electing a new leader upon killing a leader – and to see the greetings printed by the workers.
Overview: Faust vs Kafka Streams¶
KStream¶
.filter()
.filterNot()
Just use the if statement:
@app.agent(topic) async def process(stream): async for event in stream: if event.amount >= 300.0: yield event
.map()
Just call the function you want from within the
async for
iteration:@app.agent(Topic) async def process(stream): async for key, event in stream.items(): yield myfun(key, event)
.forEach()
In KS
forEach
is the same asmap
, but ends the processing chain..peek()
In KS
peek
is the same asmap
, but documents that the action may have a side effect..mapValues()
:@app.agent(topic) async def process(stream): async for event in stream: yield myfun(event)
.print()
:@app.agent(topic) async def process(stream): async for event in stream: print(event)
.writeAsText()
:@app.agent(topic) async def process(stream): async for key, event in stream.items(): with open(path, 'a') as f: f.write(repr(key, event))
.flatMap()
.flatMapValues()
@app.agent(topic) async def process(stream): async for event in stream: # split sentences into words for word in event.text.split(): yield event.derive(text=word)
.branch()
This is a special case of filter in KS, in Faust just write code and forward events as appropriate:
app = faust.App('transfer-demo') # source topic source_topic = app.topic('transfers') # destination topics tiny_transfers = app.topic('tiny_transfers') small_transfers = app.topic('small_transfers') large_transfers = app.topic('large_transfers') @app.agent(source_topic) async def process(stream): async for event in stream: if event.amount >= 1000.0: event.forward(large_transfers) elif event.amount >= 100.0: event.forward(small_transfers) else: event.forward(tiny_transfers)
.through()
:@app.agent(topic) async def process(stream): async for event in stream.through('other-topic'): yield event
.to()
:app = faust.App('to-demo') source_topic = app.topic('source') other_topic = app.topic('other') @app.agent(source_topic) async def process(stream): async for event in stream: event.forward(other_topic)
.selectKey()
Just transform the key yourself:
@app.agent(source_topic) async def process(stream): async for key, value in stream.items(): key = format_key(key)
If you want to transform the key for processors to use, then you have to change the current context to have the new key:
@app.agent(source_topic) async def process(stream): async for event in stream: event.req.key = format_key(event.req.key)
groupBy()
@app.agent(source_topic) async def process(stream): async for event in stream.group_by(Withdrawal.account): yield event
groupByKey()
???
.transform()
.transformValues()
???
.process()
Process in KS calls a Processor and is usually used to also call periodic actions (punctuation). In Faust you’d rather create a background task:
import faust # Useless example collecting transfer events # and summing them up after one second. class Transfer(faust.Record, serializer='json'): amount: float app = faust.App('transfer-demo') transfer_topic = app.topic('transfers', value_type=Transfer) class TransferBuffer: def __init__(self): self.pending = [] self.total = 0 def flush(self): for amount in self.pending: self.total += amount self.pending.clear() print('TOTAL NOW: %r' % (total,)) def add(self, amount): self.pending.append(amount) buffer = TransferBuffer() @app.agent(transfer_topic) async def task(transfers): async for transfer in transfers: buffer.add(transfer.amount) @app.timer(interval=1.0) async def flush_buffer(): buffer.flush() if __name__ == '__main__': app.main()
join()
outerJoin()
leftJoin()
NOT IMPLEMENTED
async for event in (s1 & s2).join() async for event in (s1 & s2).outer_join() async for event in (s1 & s2).left_join()
Overview: Faust vs. Celery¶
Faust is a stream processor, so what does it have in common with Celery?
If you’ve used tools such as Celery in the past, you can think of Faust as being able to, not only run tasks, but for tasks to keep history of everything that has happened so far. That is tasks (“agents” in Faust) can keep state, and also replicate that state to a cluster of Faust worker instances.
If you have used Celery you probably know tasks such as this:
from celery import Celery
app = Celery(broker='amqp://')
@app.task()
def add(x, y):
return x + y
if __name__ == '__main__':
add.delay(2, 2)
Faust uses Kafka as a broker, not RabbitMQ, and Kafka behaves differently from the queues you may know from brokers using AMQP/Redis/Amazon SQS/and so on.
Kafka doesn’t have queues, instead it has “topics” that can work pretty much the same way as queues. A topic is a log structure so you can go forwards and backwards in time to retrieve the history of messages sent.
The Celery task above can be rewritten in Faust like this:
import faust
app = faust.App('myapp', broker='kafka://')
class AddOperation(faust.Record):
x: int
y: int
@app.agent()
async def add(stream):
async for op in stream:
yield op.x + op.y
@app.command()
async def produce():
await add.send(value=AddOperation(2, 2))
if __name__ == '__main__':
app.main()
Faust also support storing state with the task (see Tables and Windowing), and it supports leader election which is useful for things such as locks.
- Learn more about Faust in the Introducing Faust introduction page
to read more about Faust, system requirements, installation instructions, community resources, and more.
- or go directly to the Quick Start tutorial
to see Faust in action by programming a streaming application.
- then explore the User Guide
for in-depth information organized by topic.
Cheat Sheet¶
Process events in a Kafka topic
orders_topic = app.topic('orders', value_serializer='json')
@app.agent(orders_topic)
async def process_order(orders):
async for order in orders:
print(order['product_id'])
Describe stream data using models
from datetime import datetime
import faust
class Order(faust.Record, serializer='json', isodates=True):
id: str
user_id: str
product_id: str
amount: float
price: float
date_created: datatime = None
date_updated: datetime = None
orders_topic = app.topic('orders', value_type=Order)
@app.agent(orders_topic)
async def process_order(orders):
async for order in orders:
print(order.product_id)
Use async. I/O to perform other actions while processing the stream
# [...]
@app.agent(orders_topic)
async def process_order(orders):
session = aiohttp.ClientSession()
async for order in orders:
async with session.get(f'http://e.com/api/{order.id}/') as resp:
product_info = await request.text()
await session.post(
f'http://cache/{order.id}/', data=product_info)
Buffer up many events at a time
Here we get up to 100 events within a 30 second window:
# [...]
async for orders_batch in orders.take(100, within=30.0):
print(len(orders))
Aggregate information into a table
orders_by_country = app.Table('orders_by_country', default=int)
@app.agent(orders_topic)
async def process_order(orders):
async for order in orders.group_by(order.country_origin):
country = order.country_origin
orders_by_country[country] += 1
print(f'Orders for country {country}: {orders_by_country[country]}')
Aggregate information using a window
Count number of orders by country, within the last two days:
orders_by_country = app.Table(
'orders_by_country',
default=int,
).hopping(timedelta(days=2))
async for order in orders_topic.stream():
orders_by_country[order.country_origin] += 1
# values in this table are not concrete! access .current
# for the value related to the time of the current event
print(orders_by_country[order.country_origin].current())
User Guide¶
- Release
1.5
- Date
Apr 17, 2019
The App - Define your Faust project¶
“I am not omniscient, but I know a lot.”
– Goethe, Faust: First part
What is an Application?¶
An application is an instance of the library, and provides the core API of Faust.
The application can define stream processors (agents), topics, channels, web views, CLI commands and more.
To create one you need to provide a name for the application (the id), a message broker, and a driver to use for table storage (optional)
>>> import faust
>>> app = faust.App('example', broker='kafka://', store='rocksdb://')
It is safe to…
Run multiple application instances in the same process:
>>> app1 = faust.App('demo1') >>> app2 = faust.App('demo2')
Share an app between multiple threads: the app is thread safe.
Application Parameters¶
You must provide a name for the app, and also you will want to
set the broker
and store
options to configure the broker URL and
a storage driver.
Other than that the rest have sensible defaults so you can safely use Faust without changing them.
Here we set the broker URL to Kafka, and the storage driver to RocksDB:
>>> app = faust.App(
... 'myid',
... broker='kafka://kafka.example.com',
... store='rocksdb://',
... )
kafka://localhost
is used if you don’t configure a broker URL.
The first part of the URL (kafka://
), is called the scheme and specifies
the driver that you want to use (it can also be the fully qualified
path to a Python class).
The storage driver decides how to keep distributed tables locally, and Faust version 1.0 supports two options:
|
RocksDB an embedded database (production) |
|
In-memory (development) |
Using the memory://
store is OK when developing your project and testing
things out, but for large tables, it can take hours to recover after
a restart, so you should never use it in production.
RocksDB recovers tables in seconds or less, is embedded and don’t require a server or additional infrastructure. It also stores them on the file system in such a way that tables can exceed the size of available memory.
See also
- Configuration Reference: for a full list of supported configuration
settings – these can be passed as keyword arguments when creating the
faust.App
.
Actions¶
app.topic()
– Create a topic-description¶
Use the topic()
method to create a topic description, used
for example to tell stream processors what Kafka topic to read from, and how the keys
and values in that topic are serialized:
topic = app.topic('name_of_topic')
@app.agent(topic)
async def process(stream):
async for event in stream:
...
key_type
/value_type
:ModelArg
Use the
key_type
andvalue_type
arguments to specify the models used for key and value serialization:class MyValueModel(faust.Record): name: str value: float topic = app.topic( 'name_of_topic', key_type=bytes, value_type=MyValueModel, )
The default
key_type
isbytes
and treats the key as a binary string. The key can also be specified as a model type (key_type=MyKeyModel
).See also
The Channels & Topics - Data Sources guide – for more about topics and channels.
The Models, Serialization, and Codecs guide – for more about models and serialization.
key_serializer
/value_serializer
:CodecArg
The codec/serializer type used for keys and values in this topic.
If not specified the default will be taken from the
key_serializer
andvalue_serializer
settings.See also
The Codecs section in the Models, Serialization, and Codecs guide – for more information on available codecs, and also how to make your own custom encoders and decoders.
partitions
:int
The number of partitions this topic should have. If not specified the default in the
topic_partitions
setting is used.Note: if this is an automatically created topic, or an externally managed source topic, then please set this value to
None
.retention
:Seconds
compacting
:bool
Set to
True
if this should be a compacting topic. The Kafka broker will then periodically compact the topic, only keeping the most recent value for a key.acks
:bool
Enable automatic acknowledgment for this topic. If you disable this then you are responsible for manually acknowledging each event.
internal
:bool
If set to
True
this means we own and are responsible for this topic: we are allowed to create or delete the topic.maxsize
:int
The maximum buffer size used for this channel, with default taken from the
stream_buffer_maxsize
setting. When this buffer is exceeded the worker will have to wait for agent/stream consumers to catch up, and if the buffer is frequently full this will negatively affect performance.Try tweaking the buffer sizes, but also the
broker_commit_interval
setting to make sure it commits more frequently with larger buffer sizes.
app.channel()
– Create a local channel¶
Use channel()
to create an in-memory communication channel:
import faust
app = faust.App('channel')
class MyModel(faust.Record):
x: int
channel = app.channel(value_type=MyModel)
@app.agent(channel)
async def process(stream):
async for event in stream:
print(f'Received: {event!r}')
@app.timer(1.0)
async def populate():
await channel.send(MyModel(303))
See also
The Channels & Topics - Data Sources guide – for more about topics and channels.
The Models, Serialization, and Codecs guide – for more about models and serialization.
key_type
/value_type
:ModelArg
Use the
key_type
andvalue_type
arguments to specify the models used for key and value serialization:class MyValueModel(faust.Record): name: str value: float channel = app.channel(key_type=bytes, value_type=MyValueModel)
key_serializer
/value_serializer
:CodecArg
The codec/serializer type used for keys and values in this channel.
If not specified the default will be taken from the
key_serializer
andvalue_serializer
settings.maxsize
:int
This is the maximum number of pending messages in the channel. If this number is exceeded any call to channel.put(value) will block until something consumes another message from the channel.
Defaults to the
stream_buffer_maxsize
setting.
app.Table()
– Define a new table¶
Use Table()
to define a new distributed dictionary; the only
required argument is a unique and identifying name. Here we also
set a default value so the table acts as a defaultdict
:
transfer_counts = app.Table(
'transfer_counts',
default=int,
key_type=str,
value_type=int,
)
The default argument is passed in as a callable, and in our example
calling int()
returns the number zero, so whenever a key is missing
in the table, it’s added with a value of zero:
>>> table['missing']
0
>>> table['also-missing'] += 1
>>> table['also-missing']
1
The table needs to relate every update to an associated source topic event,
so you must be iterating over a stream to modify a table. Like in this agent
where also .group_by()
is used to repartition the stream by account id,
ensuring every unique account delivers to the same agent instance,
and the count-per-account records accurately:
@app.agent(transfers_topic)
async def transfer(transfers):
async for transfer in transfers.group_by(Transfer.account):
transfer_counts[transfer.account] += 1
Moreover, the agent modifying the table cannot process the source topic out of
order, so only agents with concurrency=1
are allowed to update tables.
See also
The Tables and Windowing guide – for more information about tables.
Learn how to create a “windowed table” where aggregate values are placed into configurable time windows, providing you with answers to questions like “what was the value in the last five minutes”, or “what was the value of this count like yesterday”.
name
:str
The name of the table. This must be unique as two tables with the same in the same application will share changelog topics.
help
:str
Short human readable description of table purpose.
default
:Callable[[], Any]
User provided function called to get default value for missing keys.
Without any default this attempt to access a missing key will raise
KeyError
:>>> table = app.Table('nodefault', default=None) >>> table['missing'] Traceback (most recent call last): File "<stdin>", line 1, in <module> KeyError: 'missing'
With the default callback set to
int
, the same missing key will now set the key to0
and return0
:>>> table = app.Table('hasdefault', default=int) >>> table['missing'] 0
key_type
/value_type
:ModelArg
Use the
key_type
andvalue_type
arguments to specify the models used for serializing/deserializing keys and values in this table.class MyValueModel(faust.Record): name: str value: float table = app.Table(key_type=bytes, value_type=MyValueModel)
-
The name of a storage backend to use, or the URL to one.
Default is taken from the
store
setting. partitions
:int
The number of partitions for the changelog topic used by this table.
Default is taken from the
topic_partitions
setting.changelog_topic
:Topic
The changelog topic description to use for this table.
Only for advanced users who know what they’re doing.
recovery_buffer_size
:int
How often we flush changelog records during recovery. Default is every 1000 changelog messages.
standby_buffer_size
:int
How often we flush changelog records during recovery. Default is
None
(always).on_changelog_event
:Callable[[EventT], Awaitable[None]]
A callback called for every changelog event during recovery and while keeping table standbys in sync.
@app.agent()
– Define a new stream processor¶
Use the agent()
decorator to define an asynchronous stream processor:
# examples/agent.py
import faust
app = faust.App('stream-example')
@app.agent()
async def myagent(stream):
"""Example agent."""
async for value in stream:
print(f'MYAGENT RECEIVED -- {value!r}')
yield value
if __name__ == '__main__':
app.main()
This agent does not have a specific topic set – so an anonymous topic will be created for it. Use the faust agents program to list the topics used by each agent:
$ python examples/agent.py agents
┌Agents────┬───────────────────────────────────────┬────────────────┐
│ name │ topic │ help │
├──────────┼───────────────────────────────────────┼────────────────┤
│ @myagent │ stream-example-examples.agent.myagent │ Example agent. │
└──────────┴───────────────────────────────────────┴────────────────┘
The agent reads from the stream-example-examples.agent.myagent
topic, whose
name is generated from the application id
setting, the
application version
setting, and the fully qualified path of the
agent (examples.agent.myagent
).
Start a worker to consume from the topic:
$ python examples/agent.py worker -l info
Next, in a new console, send the agent a value using the faust send
program. The first argument to send is the name of the topic, and the second
argument is the value to send (use --key=k
to specify key). The name of
the topic can also start with the @ character to name an agent instead.
Use @agent
to send a value of "hello"
to the topic of our agent:
$ python examples/agent.py send @myagent hello
Finally, you should see in the worker console that it received our message:
MYAGENT RECEIVED -- b'hello'
See also
The guide-agents guide – for more information about agents.
The Channels & Topics - Data Sources guide – for more information about channels and topics.
name
:str
The name of the agent is automatically taken from the decorated function and the module it is defined in.
You can also specify the name manually, but note that this should include the module name, e.g.:
name='proj.agents.add'
.channel
:Channel
The channel or topic this agent should consume from.
concurrency
:int
The number of concurrent actors to start for this agent.
For example if you have an agent processing RSS feeds, a concurrency of
100
means you can process up to hundred RSS feeds at the same time.Adding concurrency to your agent also means it will process events in the topic out of order, and should you rewind the stream that order may differ when processing the events a second time.
Concurrency and tables
Concurrent agents are not allowed to modify tables: an exception is raised if this is attempted.
They are however allowed to read from tables.
sink
:Iterable[SinkT]
For agents that also yield a value: forward the value to be processed by one or more “sinks”.
A sink can be another agent, a topic, or a callback (async or non-async).
See also
Sinks – for more information on using sinks.
on_error
:Callable[[Agent, BaseException], None]
Optional error callback to be called when this agent raises an unexpected exception.
supervisor_strategy
:mode.SupervisorStrategyT
A supervisor strategy that decides what happens when this agent raises an exception.
The default supervisor strategy is
mode.OneForOneSupervisor
– restarting one and one agent instance as they crash.Other built-in supervisor strategies include:
-
If one agent instance of this type raises an exception we will restart all other agent instances of this type
-
If one agent instance of this type raises an exception we will crash the worker instance.
-
**kwargs
If the
channel
argument is not specified the agent will use an automatically named topic.Any additional keyword arguments are considered to be configuration for this topic, with support for the same arguments as
app.topic()
.
@app.task()
– Define a new support task.¶
Use the task()
decorator to define an asynchronous task to be started
with the app:
@app.task()
async def mytask():
print('APP STARTED AND OPERATIONAL')
The task will be started when the app starts, by scheduling it as an
asyncio.Task
on the event loop. It will only be started once the app
is fully operational, meaning it has started consuming messages from Kafka.
See also
The Tasks section in the Tasks, Timers, Cron Jobs, Web Views, and CLI Commands – for more information about defining tasks.
@app.timer()
– Define a new periodic task¶
Use the timer()
decorator to define an asynchronous periodic task
that runs every 30.0 seconds:
@app.timer(30.0)
async def my_periodic_task():
print('THIRTY SECONDS PASSED')
The timer will start 30 seconds after the worker instance has started and is in an operational state.
See also
The Timers section in the Tasks, Timers, Cron Jobs, Web Views, and CLI Commands guide – for more information about creating timers.
@app.page()
– Define a new Web View¶
Use the page()
decorator to define a new web view from an
async function:
# examples/view.py
import faust
app = faust.App('view-example')
@app.page('/path/to/view/')
async def myview(web, request):
print(f'FOO PARAM: {request.query["foo"]}')
if __name__ == '__main__':
app.main()
Next run a worker instance to start the web server on port 6066 (default):
$ python examples/view.py worker -l info
Then visit your view in the browser by going to http://localhost:6066/path/to/view/:
$ open http://localhost:6066/path/to/view/
See also
The Web Views section in the Tasks, Timers, Cron Jobs, Web Views, and CLI Commands guide – to learn more about defining views.
app.main()
– Start the faust command-line program.¶
To have your script extend the faust program, you can call
app.main()
:
# examples/command.py
import faust
app = faust.App('umbrella-command-example')
if __name__ == '__main__':
app.main()
This will use the arguments in sys.argv
and will support the same
arguments as the faust umbrella command.
To see a list of available commands, execute your program:
$ python examples/command.py
To get help for a particular subcommand run:
$ python examples/command.py worker --help
See also
The
main()
method in the API reference.
@app.command()
– Define a new command-line command¶
Use the command()
decorator to define a new subcommand
for the faust command-line program:
# examples/command.py
import faust
app = faust.App('example-subcommand')
@app.command()
async def example():
"""This docstring is used as the command help in --help."""
print('RUNNING EXAMPLE COMMAND')
if __name__ == '__main__':
app.main()
You can now run your subcommand:
$ python examples/command.py example
RUNNING EXAMPLE COMMAND
See also
The CLI Commands section in the Tasks, Timers, Cron Jobs, Web Views, and CLI Commands guide – for more information about defining subcommands.
Including how to specify command-line arguments and parameters to your command.
@app.service()
– Define a new service¶
The service()
decorator adds a custom mode.Service
class
as a dependency of the app.
What is a Service?
A service is something that can be started and stopped, and Faust is built out of many such services.
The mode library was extracted out of Faust for being generally useful, and Faust uses this library as a dependency.
Examples of classes that are services in Faust include: the
App
, a stream
, an agent
,
a table
, the TableManager
, the
Conductor
, and just about everything that is
started and stopped is.
Services can also have background tasks, or execute in an OS thread.
You can decorate a service class to have it start with the app:
# examples/service.py
import faust
from mode import Service
app = faust.App('service-example')
@app.service
class MyService(Service):
async def on_start(self):
print('MYSERVICE IS STARTING')
async def on_stop(self):
print('MYSERVICE IS STOPPING')
@Service.task
async def _background_task(self):
while not self.should_stop:
print('BACKGROUND TASK WAKE UP')
await self.sleep(1.0)
if __name__ == '__main__':
app.main()
To start the app and see it and action run a worker:
python examples/service.py worker -l info
You can also add services at runtime in application subclasses:
class MyApp(App):
def __init__(self, *args, **kwargs):
super().__init__(*args, **kwargs)
self.some_service = self.service(SomeService())
Application Signals¶
You may have experience signals in other frameworks such as Django and Celery.
The main difference between signals in Faust is that they accept
positional arguments, and that they also come with asynchronous versions
for use with asyncio
.
Signals are an implementation of the Observer design pattern.
App.on_partitions_revoked
¶
The on_partitions_revoked
signal is an asynchronous signal called after every
Kafka rebalance and provides a single argument which is the set of
newly revoked partitions.
Add a callback to be called when partitions are revoked:
from typing import Set
from faust.types import AppT, TP
@app.on_partitions_revoked.connect
async def on_partitions_revoked(app: AppT,
revoked: Set[TP], **kwargs) -> None:
print(f'Partitions are being revoked: {revoked}')
Using app
as an instance when connecting here means we will only be called
for that particular app instance. If you want to be called for all app instances
then you must connect to the signal of the class (App
):
@faust.App.on_partitions_revoked.connect
async def on_partitions_revoked(app: AppT,
revoked: Set[TP], **kwargs) -> None:
...
Signal handlers must always accept **kwargs
.
Signal handler must always accept **kwargs
so that they
are backwards compatible when new arguments are added.
Similarly new arguments must be added as keyword arguments to be backwards compatible.
App.on_partitions_assigned
¶
The on_partitions_assigned
signal is an asynchronous signal called after every
Kafka rebalance and provides a single argument which is the set of
assigned partitions.
Add a callback to be called when partitions are assigned:
from typing import Set
from faust.types import AppT, TP
@app.on_partitions_assigned.connect
async def on_partitions_assigned(app: AppT,
assigned: Set[TP], **kwargs) -> None:
print(f'Partitions are being assigned: {assigned}')
App.on_configured
¶
- sender
- arguments
- synchronous
This is a synchronous signal (do not use
async def
).
Called as the app reads configuration, just before the application configuration is set, but after the configuration is read.
Takes arguments: (app, conf)
, where conf is the faust.Settings
object being built and is the instance that app.conf
will be set to
after this signal returns.
Use the on_configured
signal to configure your app:
import os
import faust
app = faust.App('myapp')
@app.on_configured.connect
def configure(app, conf, **kwargs):
conf.broker = os.environ.get('FAUST_BROKER')
conf.store = os.environ.get('STORE_URL')
App.on_before_configured
¶
Called before the app reads configuration, and before the
App.on_configured
signal is dispatched.
Takes only sender as argument, which is the app being configured:
@app.on_before_configured.connect
def before_configuration(app, **kwargs):
print(f'App {app} is being configured')
App.on_after_configured
¶
Called after app is fully configured and ready for use.
Takes only sender as argument, which is the app that was configured:
@app.on_after_configured.connect
def after_configuration(app, **kwargs):
print(f'App {app} has been configured.')
App.on_worker_init
¶
Called by the faust worker program (or when using app.main()) to apply worker specific customizations.
Takes only sender as argument, which is the app a worker is being started for:
@app.on_worker_init.connect
def on_worker_init(app, **kwargs):
print(f'Working starting for app {app}')
Starting the App¶
You can start a worker instance for your app from the command-line, or you can start it inline in your Python process. To accommodate the many ways you may want to embed a Faust application, starting the app have several possible entry points:
App entry points:
faust worker
The faust worker program starts a worker instance for an app from the command-line.
You may turn any self-contained module into the faust program by adding this to the end of the file:
if __name__ == '__main__': app.main()
For packages you can add a
__main__.py
module or setuptools entry points tosetup.py
.If you have the module name where an app is defined, you can start a worker for it with the
faust -A
option:$ faust -A myproj worker -l info
The above will import the app from the
myproj
module usingfrom myproj import app
. If you need to specify a different attribute you can use a fully qualified path:$ faust -A myproj:faust_app worker -l info
->
faust.cli.worker.worker
(CLI interface)This is the faust worker program defined as a Python click command.
It is responsible for:
Parsing the command-line arguments supported by faust worker.
Printing the banner box (you will not get that with entry point 3 or 4).
Starting the
faust.Worker
(see next step).
->
faust.Worker
This is used for starting a worker from Python when you also want to install process signal handlers, etc. It supports the same options as on the faust worker command-line, but now they are passed in as keyword arguments to
faust.Worker
.The Faust worker is a subclass of
mode.Worker
, which makes sense given that Faust is built out of many different mode services starting in a particular order.The
faust.Worker
entry point is responsible for:Changing the directory when the
workdir
argument is set.Setting the process title (when setproctitle is installed), for more helpful entry in
ps
listings.Setting up
logging
: handlers, formatters and level.If
--debug
is enabled:Starting the aiomonitor debugging back door.
Starting the blocking detector.
Setting up
TERM
andINT
signal handlers.Setting up the
USR1
cry handler that logs a traceback.Starting the web server.
Autodiscovery (see
autodiscovery
).Starting the
faust.App
(see next step).Properly shut down of the event loop on exit.
To start a worker,
from synchronous code, use
Worker.execute_from_commandline
:>>> worker = Worker(app) >>> worker.execute_from_commandline()
or from an
async def
function callawait worker.start()
:Warning
You will be responsible for gracefully shutting down the event loop.
async def start_worker(worker: Worker) -> None: await worker.start() def manage_loop(): loop = asyncio.get_event_loop() worker = Worker(app, loop=loop) try: loop.run_until_complete(start_worker(worker)) finally: worker.stop_and_shutdown_loop()
Multiple apps
If you want your worker to start multiple apps, you would have to pass them in with the
*services
starargs:worker = Worker(app1, app2, app3, app4)
This way the extra apps will be started together with the main app, and the main app of the worker (
worker.app
) will end up being the first positional argument (app1
).Note that the web server will only concern itself with the main app, so if you want web access to the other apps you have to include web servers for them (also passed in as
*services
starargs).->
faust.App
The “worker” only concerns itself with the terminal, process signal handlers, logging, debugging mechanisms, etc., the rest is up to the app.
You can call
await app.start()
directly to get a side-effect free instance that can be embedded in any environment. It won’t even emit logs to the console unless you have configuredlogging
manually, and it won’t set up anyTERM
/INT
signal handlers, which meansfinally
blocks won’t execute at shutdown.Start app directly:
async def start_app(app): await app.start()
This will block until the worker shuts down, so if you want to start other parts of your program, you can start this in the background:
def start_in_loop(app): loop = asyncio.get_event_loop() loop.ensure_future(app.start())
If your program is written as a set of Mode services, you can simply add the app as a dependency to your service:
class MyService(mode.Service): def on_init_dependencies(self): return [faust_app]
Client-Only Mode¶
The app can also be started in “client-only” mode, which means the app can be used for sending agent RPC requests and retrieving replies, but not start a full Faust worker:
await app.start_client()
Projects and Directory Layout¶
Faust is a library; it does not mandate any specific directory layout and integrates with any existing framework or project conventions.
That said, new projects written from scratch using Faust will want some guidance on how to organize, so we include this as a suggestion in the documentation.
Small/Standalone Projects¶
You can create a small Faust service with no supporting directories at all, we refer to this as a “standalone module”: a module that contains everything it needs to run a full service.
The Faust distribution comes with several standalone examples, such as examples/word_count.py.
Medium/Large Projects¶
Projects need more organization as they grow larger, so we convert the standalone module into a directory layout:
+ proj/
- setup.py
- MANIFEST.in
- README.rst
- setup.cfg
+ proj/
- __init__.py
- __main__.py
- app.py
+ users/
- __init__.py
- agents.py
- commands.py
- models.py
- views.py
+ orders/
- __init__.py
- agents.py
- models.py
- views.py
Now we have many @app.agent/@app.timer’/@app.command decorators, and models spread across a nested directory. These have to be imported by the program to be registered and used.
Enter the autodiscover
setting:
# proj/app.py
import faust
app = faust.App(
'proj',
version=1,
autodiscover=True,
origin='proj' # imported name for this project (import proj -> "proj")
)
def main() -> None:
app.main()
Using the autodiscover
and setting it to True
means
it will traverse the directory of the origin module to find agents, timers,
tasks, commands and web views, etc.
If you want more careful control you can specify a list of modules to traverse instead:
app = faust.App(
'proj',
version=1,
autodiscover=['proj.users', 'proj.orders'],
origin='proj'
)
Autodiscovery when using Django
When using autodiscover=True in a Django project,
only the apps listed in INSTALLED_APPS
will be traversed.
See also Django Projects.
Problem: Entry Point ~~~~~~~~~~~~~~~~~~~-
The proj/__main__.py
module can act as the entry point for this
project:
# proj/__main__.py
from proj.app import app
app.main()
After creating this module you can now start a worker by doing:
python -m proj worker -l info
Now you’re probably thinking, “I’m too lazy to type python dash em all the time”, but don’t worry: take it one step further by using setuptools to install a command-line program for your project.
Create a
setup.py
for your project.This step is not needed if you already have one.
You can read lots about creating your
setup.py
in the setuptools documentation here: https://setuptools.readthedocs.io/en/latest/setuptools.html#developer-s-guideA minimum example that will work well enough:
#!/usr/bin/env python from setuptools import find_packages, setup setup( name='proj', version='1.0.0', description='Use Faust to blah blah blah', author='Ola Normann', author_email='ola.normann@example.com', url='http://proj.example.com', platforms=['any'], license='Proprietary', packages=find_packages(exclude=['tests', 'tests.*']), include_package_data=True, zip_safe=False, install_requires=['faust'], python_requires='~=3.6', )
For inspiration you can also look to the setup.py files in the faust and mode source code distributions.
Add the command as a setuptools entry point.
To your
setup.py
add the following argument:setup( ..., entry_points={ 'console_scripts': [ 'proj = proj.app:main', ], }, )
This essentially defines that the
proj
program runs from proj.app import mainInstall your package using
setup.py
or pip.When developing your project locally you should use
setup.py develop
to use the source code directory as a Python package:$ python setup.py develop
You can now run the proj command you added to
setup.py
in step two:$ proj worker -l info
Why use
develop
? You can usepython setup.py install
, but then you have to run that every time you make modifications to the source files.
Another upside to using setup.py
is that you can distribute your projects
as pip install
-able packages.
Django Projects¶
Django has their own conventions for directory layout, but your Django reusable apps will want some way to import your Faust app.
We believe the best place to define the Faust app in a Django project, is in
a dedicated reusable app. See the faustapp
app in the
examples/django
directory in the Faust source code distribution.
Miscellaneous¶
Why use applications?¶
For special needs, you can inherit from the faust.App
class, and a subclass
will have the ability to change how almost everything works.
Comparing the application to the interface of frameworks like Django, there are clear benefits.
In Django, the global settings module means having multiple configurations are impossible, and with an API organized by modules, you sometimes end up with lots of import statements and keeping track of many modules. Further, you often end up monkey patching to change how something works.
The application keeps the library flexible to changes, and allows for many applications to coexist in the same process space.
. _guide-agents:
Agents - Self-organizing Stream Processors¶
What is an Agent?¶
An agent is a distributed system processing the events in a stream.
Every event is a message in the stream and is structured as a key/value pair that can be described using models for type safety and straightforward serialization support.
Streams can be sharded in a round-robin manner, or partitioned by the message key; this decides how the stream divides between available agent instances in the cluster.
- Create an agent
To create an agent, you need to use the
@app.agent
decorator on an async function taking a stream as the argument. Further, it must iterate over the stream using theasync for
keyword to process the stream:# faustexample.py import faust app = faust.App('example', broker='kafka://localhost:9092') @app.agent() async def myagent(stream): async for event in stream: ... # process event
- Start a worker for the agent
The faust worker program can be used to start a worker from the same directory as the
faustexample.py
file:$ faust -A faustexample worker -l info
Every new worker that you start will force the cluster to rebalance partitions so that every agent receives a specific portion of the stream.
Partitioning
When an agent reads from a topic, the stream is partitioned based on the key of the message. For example, the stream could have keys that are account ids, and values that are high scores, then partitioning decide that any message with the same account id as key, is delivered to the same agent instance.
Sometimes you’ll have to repartition the stream, to ensure you are
receiving the right portion of the data. See Streams - Infinite Data Structures for
more information on the Stream.group_by()
method.
Round-Robin
If you don’t set a key (i.e. key=None
), the messages will be delivered to
available workers in round-robin order. This is useful to simply distribute work
amongst a cluster of workers, and you can always repartition that stream later
should you need to access data in a table or similar.
Fault tolerance
If the worker for a partition fails, or is blocked from the network for some reason, there is no need to worry because Kafka solves this by moving the partition to a worker that works.
Faust also takes advantage of “standby tables” and a custom partition manager that prefers to promote any node with a full copy of the data, saving startup time and ensuring availability.
Here’s a complete example of an app, having an agent that adds numbers:
# examples/agent.py
import faust
# The model describes the data sent to our agent,
# We will use a JSON serialized dictionary
# with two integer fields: a, and b.
class Add(faust.Record):
a: int
b: int
# Next, we create the Faust application object that
# configures our environment.
app = faust.App('agent-example')
# The Kafka topic used by our agent is named 'adding',
# and we specify that the values in this topic are of the Add model.
# (you can also specify the key_type if your topic uses keys).
topic = app.topic('adding', value_type=Add)
@app.agent(topic)
async def adding(stream):
async for value in stream:
# here we receive Add objects, add a + b.
yield value.a + value.b
Starting a worker will now start a single instance of this agent:
$ faust -A examples.agent worker -l info
To send values to it, open a second console to run this program:
# examples/send_to_agent.py
import asyncio
from .agent import Add, adding
async def send_value() -> None:
print(await adding.ask(Add(a=4, b=4)))
if __name__ == '__main__':
loop = asyncio.get_event_loop()
loop.run_until_complete(send_value())
$ python examples/send_to_agent.py
Define commands with the @app.command
decorator.
You can also use CLI Commands to add actions for your
application on the command line. Use the @app.command
decorator to
rewrite the example program above (examples/agent.py
), like this:
@app.command()
async def send_value() -> None:
print(await adding.ask(Add(a=4, b=4)))
After adding this to your examples/agent.py
module, run your
new command using the faust program:
$ faust -A examples.agent send_value
You may also specify command line arguments and options:
from faust.cli import argument, option
@app.command(
argument('a', type=int, help='First number to add'),
argument('b', type=int, help='Second number to add'),
option('--print/--no-print', help='Enable debug output'),
)
async def send_value(a: int, b: int, print: bool) -> None:
if print:
print(f'Sending Add({x}, {y})...')
print(await adding.ask(Add(a, b)))
Then pass those arguments on the command line:
$ faust -A examples.agent send_value 4 8 --print
Sending Add(4, 8)...
12
The Agent.ask()
method wraps the value sent in
a particular structure that includes the return address (reply-to). When the
agent sees this type of arrangement, it will reply with the result yielded
by the agent as a result of processing the event.
Static types
Faust is typed using the type annotations available in Python 3.6, and can be checked using the mypy type checker.
Add type hints to your agent function like this:
from typing import AsyncIterable
from faust import StreamT
@app.agent(topic)
async def adding(stream: StreamT[Add]) -> AsyncIterable[int]:
async for value in stream:
yield value.a + value.b
The StreamT
type used for the agent’s stream argument is a subclass
of AsyncIterable
extended with the stream API.
You could type this call using
AsyncIterable
, but then mypy would stop you with a typing
error should you use stream-specific methods such as .group_by()
,
through()
, etc.
Under the Hood: The @agent
decorator¶
You can quickly start a stream processor in Faust without using agents.
Do so merely by launching an asyncio
task that iterates over a stream:
# examples/noagents.py
import asyncio
app = faust.App('noagents')
topic = app.topic('noagents')
async def mystream():
async for event in topic.stream():
print(f'Received: {event!r}')
async def start_streams():
await app.start()
await mystream()
if __name__ == '__main__':
loop = asyncio.get_event_loop()
loop.run_until_complete(start_streams())
Essentially what the @agent
decorator does, given a function like this:
@app.agent(topic)
async def mystream(stream):
async for event in stream:
print(f'Received: {event!r}')
yield event
It wraps your function returning async iterator (since it uses
yield
) in code similar to this:
def agent(topic):
def create_agent_from(fun):
async def _start_agent():
stream = topic.stream()
async for result in fun(stream):
maybe_reply_to_caller(result)
Defining Agents¶
The Topic¶
The topic argument to the agent decorator defines the main topic that agent reads from (this implies it’s not necessarily the only topic, as is the case when using stream joins, for example).
Topics are defined using the app.topic()
helper and return a
faust.Topic
description:
topic = app.topic('topic_name1', 'topic_name2',
key_type=Model,
value_type=Model,
...)
Should the topic description provide multiple topic names, the main
topic of the agent will be the first topic in that list ("topic_name1"
).
The key_type
and value_type
describe how to serialize and deserialize
messages in the topic, and you provide it as a model (such
as faust.Record
), a faust.Codec
, or the name of a serializer.
If not specified it will use the default serializer defined by the app.
Tip
If you don’t specify a topic, the agent will use the agent name
as the topic: the name will be the fully qualified name of the agent function
(e.g., examples.agent.adder
).
See also
The Channels & Topics - Data Sources guide – for more information about topics and channels.
The Stream¶
The decorated function is unary, meaning it must accept a single argument.
The object passed in as the argument to the agent is an async iterable
Stream
instance, created from the topic/channel provided
to the decorator:
@app.agent(topic_or_channel)
async def myagent(stream):
async for item in stream:
...
Iterating over this stream, using the async for
keyword, will in turn
iterate over messages in the topic/channel.
You can also use the group_by()
method of the Stream API,
to partition the stream differently:
# examples/groupby.py
import faust
class BankTransfer(faust.Record):
account_id: str
amount: float
app = faust.App('groupby')
topic = app.topic('groupby', value_type=BankTransfer)
@app.agent(topic)
async def stream(s):
async for transfer in s.group_by(BankTransfer.account_id):
# transfers will now be distributed such that transfers
# with the same account_id always arrives to the same agent
# instance
...
A two-way join works by waiting until it has a message from both topics, so to synchronously wait for a reply from the agent you would have to send messages to both topics. A three-way join means you have to send a message to each of the three topics and only then can a reply be produced.
For this reason, you’re discouraged from using joins in an agent, unless you know what you’re doing:
topic1 = app.topic('foo1')
topic2 = app.topic('foo2')
@app.agent(topic)
async def mystream(stream):
# XXX This is not proper use of an agent, as it performs a join.
# It works fine as long as you don't expect to be able to use
# ``agent.ask``, ``agent.map`` and similar
# methods that wait for a reply.
async for event in (stream & topic2.stream()).join(...):
...
For joins, the best practice is to use the @app.task
decorator instead,
to launch an asyncio.Task
when the app starts,
that manually iterates over the joined stream:
@app.task()
def mystream():
async for event in (topic1.stream() & topic2.stream()).join(...):
# process merged event
...
See also
The Streams - Infinite Data Structures guide – for more information about streams.
The Channels & Topics - Data Sources guide – for more information about topics and channels.
Concurrency¶
Use the concurrency
argument to start multiple instances of an agent
on every worker instance. Each agent instance (actor) will process
items in the stream concurrently (and in no particular order).
Warning
Concurrent instances of an agent will process the stream out-of-order, so you cannot mutate tables from within the agent function:
An agent having concurrency > 1, can only read from a table, never write.
Here’s an agent example that can safely process the stream out of order.
Our hypothetical backend system publishes a message to the Kafka “news” topic every time a news article is published by an author.
We define an agent that consumes from this topic and for every new article will retrieve the full article over HTTP, then store that in a database somewhere (yeah, pretty contrived):
class Article(faust.Record, isodates=True):
url: str
date_published: datetime
news_topic = app.topic('news', value_type=Article)
@app.agent(news_topic, concurrency=10)
async def imports_news(articles):
async for article in articles:
response = await aiohttp.ClientSession().get(article.url)
await store_article_in_db(response)
Sinks¶
Sinks can be used to perform additional actions after the agent has processed an event in the stream, such as forwarding alerts to a monitoring system, logging to Slack, etc. A sink can be callable, async callable, a topic/channel or another agent.
- Function Callback
Regular functions take a single argument (the value yielded by the agent):
def mysink(value): print(f'AGENT YIELD: {value!r}') @app.agent(sink=[mysink]) async def myagent(stream): async for value in stream: yield value * 2
- Async Function Callback
If you provide an async function, the agent will
await
it:async def mysink(value): print(f'AGENT YIELD: {value!r}') # OBS This will force the agent instance that yielded this value # to sleep for 1.0 second before continuing on the next event # in the stream. await asyncio.sleep(1) @app.agent(sink=[mysink]) async def myagent(stream): async for value in stream: yield value * 2
- Topic
Specifying a topic as the sink will force the agent to forward yielded values it:
agent_log_topic = app.topic('agent_log') @app.agent(sink=[agent_log_topic]) async def myagent(stream): async for value in stream: yield value
- Another Agent
Specifying another agent as the sink will force the agent to forward yielded values to it:
@app.agent() async def agent_b(stream): async for value in stream: print(f'AGENT B RECEIVED: {event!r}') @app.agent(sink=[agent_b]) async def agent_a(stream): async for value in stream: print(f'AGENT A RECEIVED: {event!r}') yield value * 2
When agents raise an error¶
If an agent raises in the middle of processing an event what do we do with acking it?
Currently the source message will be acked and not processed again, simply because it violates “”exactly-once” semantics”.
What about retries?
It’d be safe to retry processing the event if the agent processing is idempotent, but we don’t enforce idempotency in stream processors so it’s not something we can enable by default.
The retry would also have to stop processing of the topic so that order is maintained: the next offset in the topic can only be processed after the event is retried.
How about crashing?
Crashing the instance to require human intervention is certainly a choice, but far from ideal considering how common mistakes in code or unexpected exceptions are. It may be better to log the error and have ops replay and reprocess the stream on notification.
Using Agents¶
Cast or Ask?¶
When communicating with an agent, you can ask for the result of the
request to be forwarded to another topic: this is the reply_to
topic.
The reply_to
topic may be the topic of another agent, a source topic
populated by a different system, or it may be a local ephemeral topic
collecting replies to the current process.
If you perform a cast
, you’re passively sending something to the agent,
and it will not reply back.
Systems perform better when no synchronization is required, so you should
try to solve your problems in a streaming manner. If B needs to happen
after A, try to have A call B instead (which could be accomplished
using reply_to=B
).
cast(value, *, key=None, partition=None)
A cast is non-blocking as it will not wait for a reply:
await adder.cast(Add(a=2, b=2))
The agent will receive the request, but it will not send a reply.
ask(value, *, key=None, partition=None, reply_to=None, correlation_id=None)
Asking an agent will send a reply back to process that sent the request:
value = await adder.ask(Add(a=2, b=2)) assert value == 4
send(key, value, partition, reply_to=None, correlation_id=None)
The
Agent.send
method is the underlying mechanism used bycast
andask
.Use it to send the reply to another agent:
await adder.send(value=Add(a=2, b=2), reply_to=another_agent)
Streaming Map/Reduce¶
These map/reduce operations are shortcuts used to stream lots of values into agents while at the same time gathering the results.
map
streams results as they come in (out-of-order), and join
waits
until all the steps are complete (back-to-order) and return the results
in a list with order preserved:
map(values: Union[AsyncIterable[V], Iterable[V]])
Map takes an async iterable, or a regular iterable, and returns an async iterator yielding results as they come in:
async for reply in agent.map([1, 2, 3, 4, 5, 6, 7, 8]): print(f'RECEIVED REPLY: {reply!r}')
The iterator will start before all the messages have been sent, and should be efficient even for infinite lists.
As the map executes concurrently, the replies will not appear in any particular order.
kvmap(items: Union[AsyncIterable[Tuple[K, V], Iterable[Tuple[K, V]]]])
Same as
map
, but takes an async iterable/iterable of(key, value)
tuples, where the key in each pair is used as the Kafka message key.join(values: Union[AsyncIterable[V], Iterable[V]])
Join works like
map
but will wait until all of the values have been processed and returns them as a list in the original order.The
await
will continue only after the map sequence is over, and all results are accounted for, so do not attempt to usejoin
together with infinite data structures ;-)results = await pow2.join([1, 2, 3, 4, 5, 6, 7, 8]) assert results == [1, 4, 9, 16, 25, 36, 49, 64]
kvjoin(items: Union[AsyncIterable[Tuple[K, V]], Iterable[Tuple[K, V]]])
Same as join, but takes an async iterable/iterable of
(key, value)
tuples, where the key in each pair is used as the message key.
Streams - Infinite Data Structures¶
“Everything transitory is but an image.”
– Goethe, Faust: Part II
Basics¶
A stream is an infinite async iterable, being passed messages consumed from a channel/topic:
@app.agent(my_topic)
async def process(stream):
async for value in s:
...
The above agent is how you usually define stream processors in Faust, but you can also create stream objects manually at any point with the caveat that this can trigger a Kafka rebalance when doing so at runtime:
stream = app.stream(my_topic) # or: my_topic.stream()
async for value in stream:
...
The stream needs to be iterated over to be processed, it will not be active until you do.
When iterated over the stream gives deserialized values, but you can also
iterate over key/value pairs (using
items()
), or raw messages (using events()
).
Keys and values can be bytes for manual deserialization, or Model
instances, and this is decided by the topic’s key_type
and
value_type
arguments.
See also
The Channels & Topics - Data Sources guide – fore more information about channels and topics.
The Models, Serialization, and Codecs guide – for more information about models and serialization.
The easiest way to process streams is to use agents, but you can also create a stream manually from any topic/channel.
Here we define a model for our stream, create a stream from the “withdrawals” topic and iterate over it:
class Withdrawal(faust.Record):
account: str
amount: float
async for w in app.topic('withdrawals', value_type=Withdrawal).stream():
print(w.amount)
Do note that the worker must be started first (or at least the app),
for this to work, and the stream iterator needs to be started
as an asyncio.Task
, so a more practical example is:
import faust
class Withdrawal(faust.Record):
account: str
amount: float
app = faust.App('example-app')
withdrawals_topic = app.topic('withdrawals', value_type=Withdrawal)
@app.task
async def mytask():
async for w in withdrawals_topic.stream():
print(w.amount)
if __name__ == '__main__':
app.main()
You may also treat the stream as a stream of bytes values:
async for value in app.topic('messages').stream():
# the topic description has no value_type, so values
# are now the raw message value in bytes.
print(repr(value))
Processors¶
A stream can have an arbitrary number of processor callbacks that are executed as values go through the stream.
These are usually not used in normal Faust applications, but can be useful for libraries to extend the functionality of streams.
A processor takes a value as argument and returns a value:
def add_default_language(value: MyModel) -> MyModel:
if not value.language:
value.language = 'US'
return value
async def add_client_info(value: MyModel) -> MyModel:
value.client = await get_http_client_info(value.account_id)
return value
s = app.stream(my_topic,
processors=[add_default_language, add_client_info])
Note
Processors can be async callable, or normal callable.
Since the processors are stored in an ordered list, the processors above will execute in order and the final value going out of the stream will be the reduction after all processors are applied:
async for value in s:
# all processors applied here so `value`
# will be equivalent to doing:
# value = add_default_language(add_client_info(value))
Message Life Cycle¶
Kafka Topics¶
Every Faust worker instance will start a single Kafka consumer responsible for fetching messages from all subscribed topics.
Every message in the topic have an offset number (where the first message has an offset of zero), and we use a single offset to track the messages that consumers do not want to see again.
The Kafka consumer commits the topic offsets every three
seconds (by default, can also be configured using the
broker_commit_interval
setting) in a background task.
Since we only have one consumer and multiple agents can be subscribed to the same topic, we need a smart way to track when those events have processed so we can commit and advance the consumer group offset.
We use reference counting for this, so when you define an agent that iterates over the topic as a stream:
@app.agent(topic)
async def process(stream):
async for value in stream:
print(value)
The act of starting that stream iterator will add the topic to the Conductor service. This internal service is responsible for forwarding messages received by the consumer to the streams:
[Consumer] -> [Conductor] -> [Topic] -> [Stream]
The async for
is what triggers this, and the agent code above
is roughly equivalent to:
async def custom_agent(app: App, topic: Topic):
topic_iterator = aiter(topic)
app.topics.add(topic) # app.topics is the Conductor
stream = Stream(topic_iterator, app=app)
async for value in stream:
print(value)
If two agents use streams subscribed to the same topic:
topic = app.topic('orders')
@app.agent(topic)
async def processA(stream):
async for value in stream:
print(f'A: {value}')
@app.agent(topic)
async def processB(stream):
async for value in stream:
print(f'B: {value}')
The Conductor will forward every message received on the “orders” topic to both of the agents, increasing the reference count whenever it enters an agents stream.
The reference count decreases when the event is acknowledged, and when it reaches zero the consumer will consider that offset as “done” and can commit it.
The acknowledgment signifies that the event processing is complete and should not happen again.
An event is automatically acknowledged when:
The agent stream advances to a new event (
Stream.__anext__
)An exception occurs in the agent during event processing.
The application shuts down, or a rebalance is required, and the stream finished processing the event.
What this means is that an event is acknowledged when your agent is finished handling it, but you can also manually control when it happens.
To manually control when the event is acknowledged, and its reference count
decreased, use await event.ack()
- async for event in stream.events():
print(event.value) await event.ack()
You can also use
async for
on the event:async for event in stream.events(): async with event: print(event.value) # event acked when exiting this block
Note that the conditions in automatic acknowledgment still apply when manually acknowledging a message.
Combining streams¶
Streams can be combined, so that you receive values from multiple streams in the same iteration:
>>> s1 = app.stream(topic1)
>>> s2 = app.stream(topic2)
>>> async for value in (s1 & s2):
... ...
Mostly this is useful when you have two topics having the same value type, but can be used in general.
If you have two streams that you want to process independently you should rather start individual tasks:
@app.agent(topic1)
async def process_stream1(stream):
async for value in stream:
...
@app.agent(topic2)
async def process_stream2(stream):
async for value in stream:
...
Operations¶
group_by()
– Repartition the stream¶
The Stream.group_by()
method repartitions the
stream by taking a “key type” as argument:
import faust
class Order(faust.Record):
account_id: str
product_id: str
amount: float
price: float
app = faust.App('group-by-example')
orders_topic = app.topic('orders', value_type=Order)
@app.agent(orders_topic)
async def process(orders):
async for order in orders.group_by(Order.account_id):
...
In the example above the “key type” is a field descriptor, and the stream will be repartitioned by the account_id field found in the deserialized stream value.
The new stream will be using a new intermediate topic where messages have account ids as key, and this is the stream that the agent will finally be iterating over.
Note
Stream.group_by()
returns a new stream subscribing to the intermediate
topic of the group by operation.
Apart from field descriptors, the key type argument can also be specified as a callable, or an async callable, so if you’re not using models to describe the data in streams you can manually extract the key used for repartitioning:
def get_order_account_id(order):
return json.loads(order)['account_id']
@app.agent(app.topic('order'))
async def process(orders):
async for order in orders.group_by(get_order_account_id):
...
See also
The Models, Serialization, and Codecs guide – for more information on field descriptors and models.
The
faust.Stream.group_by()
method in the API reference.
items()
– Iterate over keys and values¶
Use Stream.items()
to get access to both message
key and value at the same time:
@app.agent()
async def process(stream):
async for key, value in stream.items():
...
Note that this changes the type of what you iterate over from Stream
to
AsyncIterator
, so if you want to repartition the stream or similar,
.items()
need to be the last operation:
async for key, value in stream.through('foo').group_by(M.id).items():
...
events()
– Access raw messages¶
Use Stream.events()
to iterate over raw
Event
values, including access to original message payload and message
meta data:
@app.agent
async def process(stream):
async for event in stream.events():
message = event.message
topic = event.message.topic
partition = event.message.partition
offset = event.message.offset
key_bytes = event.message.key
value_bytes = event.message.value
key_deserialized = event.key
value_deserialized = event.value
async with event: # use `async with event` for manual ack
process(event)
# event will be acked when this block returns.
See also
The
faust.Event
class in the API reference – for more information about events.The
faust.types.tuples.Message
class in the API reference – for more information about the fields available inevent.message
.
take()
– Buffer up values in the stream¶
Use Stream.take()
to gather up multiple events in
the stream before processing them, for example to take 100 values at a time:
@app.agent()
async def process(stream):
async for values in stream.take(100):
assert len(values) == 100
print(f'RECEIVED 100 VALUES: {values}')
The problem with the above code is that it will block forever if there are 99 messages and the last hundredth message is never received.
To solve this add a within
timeout so that up to 100 values will be
processed within 10 seconds:
@app.agent()
async def process(stream):
async for values in stream.take(100, within=10):
print(f'RECEIVED {len(values)}: {values}')
The above code works better: if values are constantly being streamed it will process hundreds and hundreds without delay, but if there are long periods of time with no events received it will still process what it has gathered.
enumerate()
– Count values¶
Use Stream.enumerate()
to keep a count of the number of values
seen so far in a stream.
This operation works exactly like the Python enumerate()
function, but
for an asynchronous stream:
@app.agent()
async def process(stream):
async for i, value in stream.enumerate():
...
The count will start at zero by default, but enumerate
also accepts an
optional starting point argument.
See also
The
faust.utils.aiter.aenumerate()
function – for a general version ofenumerate()
that let you enumerate any async iterator, not just streams.The
enumerate()
function in the Python standard library.
through()
– Forward through another topic¶
Use Stream.through()
to forward every value
to a new topic, and replace the stream by subscribing to the new topic:
source_topic = app.topic('source-topic')
destination_topic = app.topic('destination-topic')
@app.agent()
async def process(stream):
async for value in stream.through(destination_topic):
# we are now iterating over stream(destination_topic)
print(value)
You can also specify the destination topic as a string:
# [...]
async for value in stream.through('foo'):
...
Through is especially useful if you need to convert the number of partitions in a source topic, by using an intermediate table.
If you simply want to forward a value to another topic, you can send it manually, or use the echo recipe below:
@app.agent()
async def process(stream):
async for value in stream:
await other_topic.send(value)
echo()
– Repeat to one or more topics¶
Use echo()
to repeat values received from a stream to another
channel/topic, or many other channels/topics:
@app.agent()
async def process(stream):
async for event in stream.echo('other_topic'):
...
The operation takes one or more topics, as string topic names or app.topic
, so this also works:
source_topic = app.topic('sourcetopic')
echo_topic1 = app.topic('source-copy-1')
echo_topic2 = app.topic('source-copy-2')
@app.agent(source_topic)
async def process(stream):
async for event in stream.echo(echo_topic1, echo_topic2):
...
See also
The Channels & Topics - Data Sources guide – for more information about channels and topics.
Channels & Topics - Data Sources¶
Basics¶
Faust agents iterate over streams, and streams iterate over channels.
A channel is a construct used to send and receive messages, then we have the “topic”, which is a named-channel backed by a Kafka topic.
Streams read from channels (either a local-channel or a topic).
Agent
<–> Stream
<–> Channel
Topics are named-channels backed by a transport (to use e.g. Kafka topics):
Agent
<–> Stream
<–> Topic
<–> Transport
<–> aiokafka
Faust defines these layers of abstraction so that agents can send and receive messages using more than one type of transport.
Topics are highly Kafka specific, while channels are not. That makes channels more natural to subclass should you require a different means of communication, for example using RabbitMQ (AMQP), Stomp, MQTT, NSQ, ZeroMQ, etc.
Channels¶
A channel is a buffer/queue used to send and receive messages. This buffer could exist in-memory in the local process only, or transmit serialized messages over the network.
You can create channels manually and read/write from them:
async def main():
channel = app.channel()
await channel.put(1)
async for event in channel:
print(event.value)
# the channel is infinite so we break after first event
break
Reference¶
-
class
faust.
Channel
[source] -
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source] Send message to channel.
- Return type
-
as_future_message
(key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → faust.types.tuples.FutureMessage[source] - Return type
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source] - Return type
-
coroutine
Note
Some channels may require you to declare them on the server side before they’re used. Faust will create topics considered internal but will not create or modify “source topics” (i.e., exposed for use by other Kafka applications).
To define a topic as internal use
app.topic('name', ..., internal=True)
.
Models, Serialization, and Codecs¶
Basics¶
Models describe the fields of data structures used as keys and values
in messages. They’re defined using a NamedTuple
-like syntax,
as introduced by Python 3.6, and look like this:
class Point(Record, serializer='json'):
x: int
y: int
Here we define a “Point” record having x
, and y
fields of type int.
There’s no type checking at runtime, but the mypy type checker can be used
as a separate build step to verify that arguments have the correct type.
A record is a model of the dictionary type, and describes keys and values. When using JSON as the serialization format, the Point model above serializes as:
>>> Point(x=10, y=100).dumps()
{"x": 10, "y": 100}
A different serializer can be provided as an argument to .dumps
:
>>> Point(x=10, y=100).dumps('pickle') # pickle + Base64
b'gAN9cQAoWAEAAAB4cQFLClgBAAAAeXECS2RYBwAAAF9fZmF1c3RxA31xBFg
CAAAAbnNxBVgOAAAAX19tYWluX18uUG9pbnRxBnN1Lg=='
“Record” is the only model type supported by this version of Faust, but is just one of many possible model types to include in the future. The Avro serialization schema format, which the terminology is taken from, supports records, arrays, and more.
Manual Serialization¶
You’re not required to define models to read the data from a stream.
Manual de-serialization also works and is rather easy to perform.
Models provide additional benefit, such as the field descriptors that
let you refer to fields in group_by statements, static typing using
mypi, automatic conversion of datetime
, and so on…
To deserialize streams manually, merely use a topic with bytes values:
topic = app.topic('custom', value_type=bytes)
@app.agent
async def processor(stream):
async for payload in stream:
data = json.loads(payload)
To integrate with external systems, Faust’s Codecs can help you support serialization and de-serialization to and from any format. Models describe the form of messages, while codecs explain how they’re serialized/compressed/encoded/etc.
The default codec is configured by the applications key_serializer
and
value_serializer
arguments:
app = faust.App(key_serializer='json')
Individual models can override the default
by specifying a serializer
argument when creating the model class:
class MyRecord(Record, serializer='json'):
...
Codecs can also be combined, so they consist of multiple encoding and decoding
stages, for example, data serialized with JSON and then Base64 encoded would
be described as the keyword argument serializer='json|binary'
.
See also
The Codecs section – for more information about codecs and how to define your own.
Sending/receiving raw values
Serializing/deserializing keys and values manually without models is easy.
The JSON codec happily accepts lists and dictionaries,
as arguments to the .send
methods:
# examples/nondescript.py
import faust
app = faust.App('values')
transfers_topic = app.topic('transfers')
large_transfers_topic = app.topic('large_transfers')
@app.agent(transfers_topic)
async def find_large_transfers(transfers):
async for transfer in transfers:
if transfer['amount'] > 1000.0:
await large_transfers_topic.send(value=transfer)
async def send_transfer(account_id, amount):
await transfers_topic.send(value={
'account_id': account_id,
'amount': amount,
})
Using models to describe topics provides benefits:
# examples/described.py
import faust
class Transfer(faust.Record):
account_id: str
amount: float
app = faust.App('values')
transfers_topic = app.topic('transfers', value_type=Transfer)
large_transfers_topic = app.topic('large_transfers', value_type=Transfer)
@app.agent(transfers_topic)
async def find_large_transfers(transfers):
async for transfer in transfers:
if transfer.amount > 1000.0:
await large_transfers_topic.send(value=transfer)
async def send_transfer(account_id, amount):
await transfers_topic.send(
value=Transfer(account_id=account_id, amount=amount),
)
The mypy static type analyzer can now alert you if your
code is passing the wrong type of value for the account_id
field,
and more. The most compelling reason for using non-described messages would
be to integrate with existing Kafka topics and systems, but if you’re
writing new systems in Faust, the best practice would be to describe models
for your message data.
Model Types¶
The first version of Faust only supports dictionary models (records), but can be easily extended to support other types of models, like arrays.
Records¶
A record is a model based on a dictionary/mapping. The storage
used is a dictionary, and it serializes to a dictionary, but the same
is true for ordinary Python objects and their __dict__
storage, so you can
consider record models to be “objects” that can have methods and properties.
Here’s a simple record describing a 2d point, with two required fields: x
and y
:
class Point(faust.Record):
x: int
y: int
To create a new point, instantiate it like a regular Python object, and provide fields as keyword arguments:
>>> point = Point(x=10, y=20)
>>> point
<Point: x=10, y=20>
Faust throws an error if you instantiate a model without providing values for all required fields:
>>> point = Point(x=10)
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/opt/devel/faust/faust/models/record.py", line 96, in __init__
self._init_fields(fields)
File "/opt/devel/faust/faust/models/record.py", line 106, in _init_fields
type(self).__name__, ', '.join(sorted(missing))))
TypeError: Point missing required arguments: y
Note
Python does not check types at runtime. The annotations are only used by static analysis tools like mypy.
To describe an optional field, provide a default value:
class Point(faust.Record, serializer='json'):
x: int
y: int = 0
You can now omit the y
field when creating a new point:
>>> point = Point(x=10)
>>> point
<Point: x=10, y=0>
When sending messages to topics, we can use Point
objects as message keys,
and values:
await app.send('mytopic', key=Point(x=10, y=20), value=Point(x=30, y=10))
The above will send a message to Kafka, and whatever receives that message must be able to deserialize the data.
To define an agent that is able to do so, define the topic to have specific key/value types:
my_topic = faust.topic('mytopic', key_type=Point, value_type=Point)
@app.agent(my_topic)
async def task(events):
async for event in events:
print(event)
Warning
You need to restart all Faust instances using the old key/value types, or alternatively provide an upgrade path for old instances.
Records can also have other records as fields:
class Account(faust.Record, serializer='json'):
id: str
balance: float
class Transfer(faust.Record, serializer='json'):
account: Account
amount: float
transfer = Transfer(
account=Account(id='RBH1235678', balance=13000.0),
amount=1000.0,
)
To manually serialize a record use its .dumps()
method:
>>> json = transfer.dumps()
To convert the JSON back into a model use the .loads()
class method:
>>> transfer = Transfer.loads(json_bytes_data)
Records can also have fields that are a list of other models, or mappings to other models, and these are also described using the type annotation syntax.
To define a model that points to a list of Account objects you can do this:
from typing import List
import faust
class LOL(faust.Record):
accounts: List[Account]
This works with many of the iterable types, so for a list all
of Sequence
, MutableSequence
, and List
can be used. For a full list of generic data types
recognized by Faust, consult the following table:
Collection |
Recognized Annotations |
---|---|
List |
|
Set |
|
Tuple |
|
Mapping |
From this table we can see that we can also have a mapping of username to account:
from typing import Mapping
class DOA(faust.Record):
accounts: Mapping[str, Account]
Faust will automatically reconstruct the DOA.accounts
field into
a mapping of string to Account
objects.
There are limitations to this, and Faust may not recognize your custom mapping or list type, so stick to what is listed in the table for your Faust version.
Faust automatically serializes datetime
fields to
ISO-8601 text format but will not automatically deserialize ISO-8601 strings
back into datetime
(it is impossible to distinguish them
from ordinary strings).
However, if you use a model with a datetime
field, and enable the
isodates
model class setting, the model will correctly convert the strings
to datetime objects (with timezone information if available) when
deserialized:
from datetime import datetime
import faust
class Account(faust.Record, isodates=True, serializer='json'):
date_joined: datetime
Similar to datetimes, json does not have a suitable high precision decimal field type.
You can enable the decimals=True
option to coerce string decimal
values back into Python decimal.Decimal
objects.
from decimal import Decimal
import faust
class Order(faust.Record, decimals=True, serializer='json'):
price: Decimal
quantity: Decimal
You can add custom coercion rules to your model classes
using the coercions
options. This must be a mapping from, either a tuple
of types or a single type, to a function/class/callable used to convert it.
Here’s an example converting strings back to UUID objects:
from uuid import UUID
import faust
class Account(faust.Record, coercions={UUID: UUID}):
id: UUID
You’d get tired writing this out for every class, so why not make an abstract model subclass:
from uuid import UUID
import faust
class UUIDAwareRecord(faust.Record,
abstract=True,
coercions={UUID: UUID}):
...
class Account(UUIDAwareRecord):
id: UUID
You can mark a model class as abstract=True
to create a model base class,
that you must inherit from to create new models having common functionality.
For example, you may want to have a base class for all models that have fields for time of creation, and time last created.
class MyBaseRecord(Record, abstract=True):
time_created: float = None
time_updated: float = None
An “abstract” class is only used to create new models:
class Account(MyBaseRecord):
id: str
account = Account(id='X', time_created=3124312.3442)
print(account.time_created)
You can also create model values using positional arguments,
meaning that Point(x=10, y=30)
can also be expressed as Point(10, 30)
.
The ordering of fields in positional arguments gets tricky when you add subclasses to the mix. In that case, the ordering is decided by the method resolution order, as demonstrated by this example:
import faust
class Point(faust.Record):
x: int
y: int
class XYZPoint(Point):
z: int
point = XYZPoint(10, 20, 30)
assert (point.x, point.y, point.z) == (10, 20, 30)
Blessed Keys and polymorphic fields¶
Models can contain fields that are other models, such as in this example where an account has a user:
class User(faust.Record):
id: str
first_name: str
last_name: str
class Account(faust.Record, decimals=True):
user: User
balance: Decimal
This is a strict relationship, the value for Account.user can only
ever be a User
class.
Faust records also support polymorphic fields, where the type of the field is decided at runtime. Consider an Article model having a list of assets:
class Asset(faust.Record):
url: str
type: str
class ImageAsset(faust.Record):
type = 'image'
class VideoAsset(faust.Record):
runtime_seconds: float
type = 'video'
class Article(faust.Record, allow_blessed_key=True):
assets: List[Asset]
How does this work? What is a blessed key? The answer is in how Faust models are serialized and deserialized.
When serializing a Faust model we always add a special key, let’s look at the Account object we defined above, and how the payloads are generated:
>>> user = User(
... id='07ecaebf-48c4-4c9e-92ad-d16d2f4a9a19',
... first_name='Franz',
... last_name='Kafka',
... )
>>> account = Account(
... user=user,
... balance='12.3',
)
>>> from pprint import pprint
>>> pprint(account.to_representation())
{
'__faust': {'ns': 't.Account'},
'balance': Decimal('12.3'),
'user': {
'__faust': {'ns': 't.User'},
'first_name': 'Franz',
'id': '07ecaebf-48c4-4c9e-92ad-d16d2f4a9a19',
'last_name': 'Kafka',
},
}
The blessed key here is the __faust
key, it describes what model
class was used when serializing it. When we allow the blessed key to be used,
we allow it to be reconstructed using that same class.
When you define a module in Python code, Faust will automatically keep an index of model name to class, which we then use to look up a model class by name. For this to work, you must have imported the module where your model is defined before you deserialize the payload.
When using blessed keys it’s extremely important that you do not rename classes, or old data cannot be deserialized.
-
class
faust.
Record
[source] -
classmethod
loads
(s: bytes, *, default_serializer: Union[faust.types.codecs.CodecT, str, None] = None, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → faust.types.models.ModelT Deserialize model object from bytes.
- Keyword Arguments
serializer (CodecArg) – Default serializer to use if no custom serializer was set for this model subclass.
- Return type
-
dumps
(*, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → bytes Serialize object to the target serialization format.
- Return type
-
to_representation
() → Mapping[str, Any][source] Convert object to JSON serializable object.
-
classmethod
from_data
(data: Mapping, *, preferred_type: Type[faust.types.models.ModelT] = None) → faust.models.record.Record[source] - Return type
-
derive
(*objects, **fields) → faust.types.models.ModelT - Return type
-
_options
Model metadata for introspection. An instance of
faust.types.models.ModelOptions
.
-
classmethod
-
class
faust.
ModelOptions
[source] -
fields
= None Flattened view of __annotations__ in MRO order.
- Type
Index
-
fieldset
= None Set of required field names, for fast argument checking.
- Type
Index
-
fieldpos
= None Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.
- Type
Index
-
optionalset
= None Set of optional field names, for fast argument checking.
- Type
Index
-
models
= None Mapping of fields that are ModelT
- Type
Index
-
decimals
= False
-
isodates
= False
-
coercions
= None
-
defaults
= None Mapping of field names to default value.
-
Codecs¶
Supported codecs¶
raw - no encoding/serialization (bytes only).
json -
json
with UTF-8 encoding.pickle -
pickle
with Base64 encoding (not URL-safe).binary - Base64 encoding (not URL-safe).
Encodings are not URL-safe if the encoded payload cannot be embedded directly into a URL query parameter.
Serialization by name¶
The dumps()
function takes a codec name and the object to encode as arguments,
and returns bytes
>>> s = dumps('json', obj)
In reverse direction, the loads()
function takes a codec name and
an encoded payload to decode (in bytes), as arguments, and returns a
reconstruction of the serialized object:
>>> obj = loads('json', s)
When passing in the codec type as a string (as in loads('json', ...)
above), you can also
combine multiple codecs to form a pipeline, for example "json|gzip"
combines JSON
serialization with gzip compression:
>>> obj = loads('json|gzip', s)
Codec registry¶
All codecs have a name and the faust.serializers.codecs
attribute
maintains a mapping from name to Codec
instance.
You can add a new codec to this mapping by executing:
>>> from faust.serializers import codecs
>>> codecs.register(custom, custom_serializer())
To create a new codec, you need to define only two methods: first
you need the _loads()
method to deserialize bytes, then you need
the _dumps()
method to serialize an object:
import msgpack
from faust.serializers import codecs
class raw_msgpack(codecs.Codec):
def _dumps(self, obj: Any) -> bytes:
return msgpack.dumps(obj)
def _loads(self, s: bytes) -> Any:
return msgpack.loads(s)
We use msgpack.dumps
to serialize, and our codec now encodes
to raw msgpack format in binary. We may have to write
this payload to somewhere unable to handle binary data well,
to solve that we combine the codec with Base64 encoding to convert
the binary to text.
Combining codecs is easy using the |
operator:
def msgpack() -> codecs.Codec:
return raw_msgpack() | codecs.binary()
codecs.register('msgpack', msgpack())
At this point, we monkey-patched Faust to support our codec, and we can use it to define records:
>>> from faust import Record
>>> class Point(Record, serializer='msgpack'):
... x: int
... y: int
The problem with monkey-patching is that we must make sure the patching happens before we use the feature.
Faust also supports registering codec extensions using setuptools entry-points, so instead, we can create an installable msgpack extension.
To do so, we need to define a package with the following directory layout:
faust-msgpack/
setup.py
faust_msgpack.py
The first file (faust-msgpack/setup.py
) defines metadata about our
package and should look like the following example:
import setuptools
setuptools.setup(
name='faust-msgpack',
version='1.0.0',
description='Faust msgpack serialization support',
author='Ola A. Normann',
author_email='ola@normann.no',
url='http://github.com/example/faust-msgpack',
platforms=['any'],
license='BSD',
packages=find_packages(exclude=['ez_setup', 'tests', 'tests.*']),
zip_safe=False,
install_requires=['msgpack-python'],
tests_require=[],
entry_points={
'faust.codecs': [
'msgpack = faust_msgpack:msgpack',
],
},
)
The most important part being the entry_points
key which tells
Faust how to load our plugin. We have set the name of our
codec to msgpack
and the path to the codec class
to be faust_msgpack:msgpack
. Faust imports this as it would
from faust_msgpack import msgpack
, so we need to define
that part next in our faust-msgpack/faust_msgpack.py
module:
from faust.serializers import codecs
class raw_msgpack(codecs.Codec):
def _dumps(self, obj: Any) -> bytes:
return msgpack.dumps(s)
def msgpack() -> codecs.Codec:
return raw_msgpack() | codecs.binary()
That’s it! To install and use our new extension do:
$ python setup.py install
At this point you can publish this to PyPI so it can be shared amongst other Faust users.
Tables and Windowing¶
“A man sees in the world what he carries in his heart.”
– Goethe, Faust: First part
Tables¶
Basics¶
A table is a distributed in-memory dictionary, backed by a Kafka changelog topic used for persistence and fault-tolerance. We can replay the changelog upon network failure and node restarts, allowing us to rebuild the state of the table as it was before the fault.
To create a table use app.Table
:
table = app.Table('totals', default=int)
You cannot modify a table outside of a stream operation; this means that you can
only mutate the table from within an async for event in stream:
block.
We require this to align the table’s partitions with the stream’s, and to
ensure the source topic partitions are correctly rebalanced to a different
worker upon failure, along with any necessary table partitions.
Modifying a table outside of a stream will raise an error:
totals = app.Table('totals', default=int)
# cannot modify table, as we are not iterating over stream
table['foo'] += 30
This source-topic-event to table-modification-event requirement also ensures that producing to the changelog and committing messages from the source happen simultaneously.
Warning
An abruptly terminated Faust worker can allow some changelog entries to go through, before having committed the source topic offsets.
Duplicate messages may result in double-counting and other data consistency issues, but since version 1.5 of Faust you can enable a setting for strict processing guarantees.
See the processing_guarantee
setting for more information.
Co-partitioning Tables and Streams¶
When managing stream partitions and their corresponding changelog partitions, “co-partitioning” ensures the correct distribution of stateful processing among available clients, but one requirement is that tables and streams must share shards.
To shard the table differently, you must first repartition the stream using
group_by
.
Repartition a stream:
withdrawals_topic = app.topic('withdrawals', value_type=Withdrawal)
country_to_total = app.Table(
'country_to_total', default=int).tumbling(10.0, expires=10.0)
withdrawals_stream = app.topic('withdrawals', value_type=Withdrawal).stream()
withdrawals_by_country = withdrawals_stream.group_by(Withdrawal.country)
@app.agent
async def process_withdrawal(withdrawals):
async for withdrawal in withdrawals.group_by(Withdrawal.country):
country_to_total[withdrawal.country] += withdrawal.amount
If the stream and table are not co-partitioned, we could end up with a table shard ending up on a different worker than the worker processing its corresponding stream partition.
Warning
For this reason, table changelog topics must have the same number of partitions as the source topic.
Table Sharding¶
Tables shards in Kafka must organize using a disjoint distribution of keys so that any computation for a subset of keys always happen together in the same worker process.
The following is an example of incorrect usage where subsets of keys are likely to be processed by different worker processes:
withdrawals_topic = app.topic('withdrawals', key_type=str,
value_type=Withdrawal)
user_to_total = app.Table('user_to_total', default=int)
country_to_total = app.Table(
'country_to_total', default=int).tumbling(10.0, expires=10.0)
@app.agent(withdrawals_topic)
async def process_withdrawal(withdrawals):
async for withdrawal in withdrawals:
user_to_total[withdrawal.user] += withdrawal.amount
country_to_total[withdrawal.country] += withdrawal.amount
Here the stream withdrawals
is (implicitly) partitioned by the user ID used
as message key. So the country_to_total
table, instead of being
partitioned by country name, is partitioned by the user ID. In practice,
this means that data for a country may reside on multiple partitions, and
worker instances end up with incomplete data.
To fix that rewrite your program like this, using two distinct agents and repartition the stream by country when populating the table:
withdrawals_topic = app.topic('withdrawals', value_type=Withdrawal)
user_to_total = app.Table('user_to_total', default=int)
country_to_total = app.Table(
'country_to_total', default=int).tumbling(10.0, expires=10.0)
@app.agent(withdrawals_topic)
async def find_large_user_withdrawals(withdrawals):
async for withdrawal in withdrawals:
user_to_total[withdrawal.user] += withdrawal.amount
@app.agent(withdrawals_topic)
async def find_large_country_withdrawals(withdrawals):
async for withdrawal in withdrawals.group_by(Withdrawal.country):
country_to_total[withdrawal.country] += withdrawal.amount
The Changelog¶
Every modification to a table has a corresponding changelog update, the changelog is used to recover data after a failure.
We store the changelog in Kafka as a topic and use log compaction to only keep the most recent value for a key in the log. Kafka periodically compacts the table, to ensure the log does not grow beyond the number of keys in the table.
Note
In production the RocksDB store allows for almost instantaneous recovery of tables: a worker only needs to retrieve updates missed since last time the instance was up.
If you change the value for a key in the table, please make sure you update the table with the new value after:
In order to publish a changelog message into Kafka for fault-tolerance the table needs to be set explicitly. Hence, while changing values in Tables by reference, we still need to explicitly set the value to publish to the changelog, as shown below:
user_withdrawals = app.Table('user_withdrawals', default=list)
topic = app.topic('withdrawals', value_type=Withdrawal)
async for event in topic.stream():
# get value for key in table
withdrawals = user_withdrawals[event.account]
# modify the value
withdrawals.append(event.amount)
# write it back to the table (also updating changelog):
user_withdrawals[event.account] = withdrawals
If you forget to do so, like in the following example, the program will work but will have inconsistent data if a recovery is needed for any reason:
user_withdrawals = app.Table('user_withdrawals', default=list)
topic = app.topic('withdrawals', value_type=Withdrawal)
async for event in topic.stream():
withdrawals = user_withdrawals[event.account]
withdrawals.append(event.amount)
# OOPS! Did not update the table with the new value
Due to this changelog, both table keys and values must be serializable.
See also
The Models, Serialization, and Codecs guide for more information about models and serialization.
Note
Faust creates an internal changelog topic for each table. The Faust application should be the only client producing to the changelog topics.
Windowing¶
Windowing allows us to process streams while preserving state over defined windows of time. A windowed table preserves key-value pairs according to the configured “Windowing Policy.”
We support the following policies:
-
class
TumblingWindow
¶
This class creates fixed-sized, non-overlapping and contiguous time intervals
to preserve key-value pairs, e.g. Tumbling(10)
will create non-overlapping
10 seconds windows:
window 1: ----------
window 2: ----------
window 3: ----------
window 4: ----------
window 5: ----------
This class is exposed as a method from the output of app.Table()
, it takes
a mandatory parameter size
, representing the window (time interval) duration
and an optional parameter expires
, representing the duration for which we
want to store the data (key-value pairs) allocated to each window.
-
class
HoppingWindow
¶
This class creates fixed-sized, overlapping time intervals to preserve key-value
pairs, e.g. Hopping(10, 5)
will create overlapping 10 seconds windows. Each
window will be created every 5 seconds.
window 1: ----------
window 2: ----------
window 3: ----------
window 4: ----------
window 5: ----------
window 6: ----------
This class is exposed as a method from the output of app.Table()
, it takes 2
mandatory parameters:
size
, representing the window (time interval) duration.step
, representing the time interval used to create new windows.
It also takes an optional parameter expires
, representing the duration for
which we want to store the data (key-value pairs) allocated to each window.
How To¶
You can define a windowed table like this:
from datetime import timedelta
views = app.Table('views', default=int).tumbling(
timedelta(minutes=1),
expires=timedelta(hours=1),
)
Since a key can exist in multiple windows, the windowed table returns a special
wrapper for table[k]
, called a WindowSet
.
Here’s an example of a windowed table in use:
page_views_topic = app.topic('page_views', value_type=str)
@app.agent(events_topic)
async def aggregate_page_views(pages):
# values in this streams are URLs as strings.
async for page_url in pages:
# increment one to all windows this page URL fall into.
views[page_url] += 1
if views[page_url].now() >= 10000:
# Page is trending for current processing time window
print('Trending now')
if views[page_url].current() >= 10000:
# Page would be trending in the current event's time window
print('Trending when event happened')
if views[page_url].value() >= 10000:
# Page would be trending in the current event's time window
# according to the relative time set when creating the
# table.
print('Trending when event happened')
if views[page_url].delta(timedelta(minutes=30)) > views[page_url].now():
print('Less popular compared to 30 minutes back')
In this table, table[k].now()
returns the most recent value for the
current processing window, overriding the _relative_to_ option used to create
the window.
In this table, table[k].current()
returns the most recent value relative
to the time of the currently processing event, overriding the _relative_to_
option used to create the window.
In this table, table[k].value()
returns the most recent value relative
to the time of the currently processing event, and is the default behavior.
You can also make the current value relative to the current local time, relative to a different field in the event (if it has a custom timestamp field), or of another event.
The default behavior is “relative to current stream”:
views = app.Table('views', default=int).tumbling(...).relative_to_stream()
Where .relative_to_stream()
means values are selected based on the window
of the current event in the currently processing stream.
You can also use .relative_to_now()
: this means the window of the current
local time is used instead:
views = app.Table('views', default=int).tumbling(...).relative_to_now()
If the current event has a custom timestamp field that you want to use,
relative_to_field(field_descriptor)
is suited for that task:
views = app.Table('views', default=int) \
.tumbling(...) \
.relative_to_field(Account.date_created)
You can override this default behavior when accessing data in the table:
@app.agent(topic)
async def process(stream):
async for event in stream:
# Get latest value for key', based on the tables default
# relative to option.
print(table[key].value())
# You can bypass the default relative to option, and
# get the value closest to the event timestamp
print(table[key].current())
# You can bypass the default relative to option, and
# get the value closest to the current local time
print(table[key].now())
# Or get the value for a delta, e.g. 30 seconds ago, relative
# to the event timestamp
print(table[key].delta(30))
Note
We always retrieve window data based on timestamps. With tumbling windows there is just one window at a time, so for a given timestamp there is just one corresponding window. This is not the case for for hopping windows, in which a timestamp could be located in more than 1 window.
At this point, when accessing data from a hopping table, we always access the latest window for a given timestamp and we have no way of modifying this behavior.
Iterating over keys/values/items in a windowed table.¶
Note
Tables are distributed across workers, so when iterating over table contents you will only see the partitions assigned to the current worker.
Iterating over all the keys in a table will require you to visit all workers, which is highly impractical in a production system.
For this reason table iteration is mostly used in debugging and observing your system.
To iterate over the keys/items/values in windowed table you may
add the key_index
option to enable support for it:
windowed_table = app.Table(
'name',
default=int,
).hopping(10, 5, expires=timedelta(minutes=10), key_index=True)
Adding the key index means we keep a second table as an index of the keys present in the table. Whenever a new key is added we add the key to the key index, similarly whenever a key is deleted we also delete it from the index.
This enables fast iteration over the keys, items and values in the windowed table, with the caveat that those keys may not exist in all windows.
The table iterator views (.keys()
/.items()
/.values()
)
will be time-relative to the stream by default, unless you have changed
the time-relativity using the .relative_to_now
or
relative_to_timestamp
modifiers:
# Show keys present relative to time of current event in stream:
print(list(windowed_table.keys()))
# Show items present relative to time of current event in stream:
print(list(windowed_table.items()))
# Show values present relative to time of current event in stream:
print(list(windowed_table.values()))
You can also manually specify the time-relativity:
# Change time-relativity to current wall-clock time,
# and show a list of items present in that window.
print(list(windowed_table.relative_to_now().items()))
# Get items present 30 seconds ago:
print(list(windowed_table.relative_to_now().items().delta(30.0)))
“Out of Order” Events¶
Kafka maintains the order of messages published to it, but when using custom timestamp fields, relative ordering is not guaranteed.
For example, a producer can lose network connectivity while sending a batch of messages and be forced to retry sending them later, then messages in the topic won’t be in timestamp order.
Windowed tables in Faust correctly handles such “out of order ” events, at least until the message is as old as the table expiry configuration.
Note
We handle out of order events by storing separate aggregates for each
window in the last expires
seconds. The space complexity for this
is O(w * K)
where w
is the number of windows in the last
expires seconds and K
is the number of keys in the table.
Tasks, Timers, Cron Jobs, Web Views, and CLI Commands¶
Tasks¶
Your application will have agents that process events in streams, but
can also start asyncio.Task
-s that do other things,
like periodic timers, views for the embedded web server, or additional
command-line commands.
Decorating an async function with the @app.task decorator will tell the worker to start that function as soon as the worker is fully operational:
@app.task
async def on_started():
print('APP STARTED')
If you add the above to the module that defines your app and start the worker, you should see the message printed in the output of the worker.
A task is a one-off task; if you want to do something at periodic intervals, you can use a timer.
Timers¶
A timer is a task that executes every n
seconds:
@app.timer(interval=60.0)
async def every_minute():
print('WAKE UP')
After starting the worker, and it’s operational, the above timer will print something every minute.
Cron Jobs¶
A Cron job is a task that executes according to a Crontab format, usually at fixed times:
@app.crontab('0 20 * * *')
async def every_dat_at_8_pm():
print('WAKE UP ONCE A DAY')
After starting the worker, and it’s operational, the above Cron job will print something every day at 8pm.
crontab
takes 1 mandatory argument cron_format
and 2 optional arguments:
tz
, represents the timezone. Defaults to None which gives behaves as UTC.on_leader
, boolean defaults to False, only run on leader?
@app.crontab('0 20 * * *', tz=pytz.timezone('US/Pacific'), on_leader=True)
async def every_dat_at_8_pm_pacific():
print('WAKE UP AT 8:00pm PACIFIC TIME ONLY ON THE LEADER WORKER')
Web Views¶
The Faust worker will also expose a web server on every instance, that by default runs on port 6066. You can access this in your web browser after starting a worker instance on your local machine:
$ faust -A myapp worker -l info
Just point your browser to the local port to see statistics about your running instance:
http://localhost:6066
You can define additional views for the web server (called pages). The server will use the aiohttp HTTP server library, but you can also write custom web server drivers.
Add a simple page returning a JSON structure by adding this to your app module:
# this counter exists in-memory only,
# so will be wiped when the worker restarts.
count = [0]
@app.page('/count/')
async def get_count(self, request):
# update the counter
count[0] += 1
# and return it.
return self.json({
'count': count[0],
})
This example view is of limited usefulness. It only provides you with a count of how many times the page is requested, on that particular server, for as long as it’s up, but you can also call actors or access table data in web views.
Restart your Faust worker, and you can visit your new page at:
http://localhost:6066/count/
Your workers may have an arbitrary number of views, and it’s up to you what they provide. Just like other web applications they can communicate with Redis, SQL databases, and so on. Anything you want, really, and it’s executing in an asynchronous event loop.
You can decide to develop your web app directly in the Faust workers, or you may choose to keep your regular web server separate from your Faust workers.
You can create complex systems quickly, just by putting everything in a single Faust app.
HTTP Verbs: GET
/POST
/PUT
/DELETE
¶
Specify a faust.web.View
class when you need to handle HTTP
verbs other than GET
:
from faust.web import Request, Response, View
@app.page('/count/')
class counter(View):
count: int = 0
async def get(self, request: Request) -> Response
return self.json({'count': self.count})
async def post(self, request: Request) -> Response:
n: int = request.query['n']
self.count += 1
return self.json({'count': self.count})
async def delete(self, request: Request) -> Response:
self.count = 0
Exposing Tables¶
A frequent requirement is the ability to expose table values in a web view, and while this is likely to be built-in to Faust in the future, you will have to implement this manually for now.
Tables are partitioned by key, and data for any specific key will exist
on a particular worker instance. You can use the @app.table_route
decorator to reroute the request to the worker holding that partition.
We define our table, and an agent reading from the stream to populate the table:
import faust
app = faust.App(
'word-counts',
broker='kafka://localhost:9092',
store='rocksdb://',
topic_partitions=8,
)
posts_topic = app.topic('posts', value_type=str)
word_counts = app.Table('word_counts', default=int,
help='Keep count of words (str to int).')
class Word(faust.Record):
word: str
@app.agent(posts_topic)
async def shuffle_words(posts):
async for post in posts:
for word in post.split():
await count_words.send(key=word, value=Word(word=word))
@app.agent()
async def count_words(words):
"""Count words from blog post article body."""
async for word in words:
word_counts[word.word] += 1
After that we define the view, using the @app.table_route
decorator to
reroute the request to the correct worker instance:
@app.page('/count/{word}/')
@app.table_route(table=word_counts, match_info='word')
async def get_count(web, request, word):
return web.json({
word: word_counts[word],
})
In the above example we used part of the URL to find the given word, but you may also want to get this from query parameters.
Table route based on key in query parameter:
@app.page('/count/')
@app.table_route(table=word_counts, query_param='word')
async def get_count(web, request):
word = request.query['word']
return web.json({
word: word_counts[word],
})
CLI Commands¶
As you may already know, you can make your project into an executable,
that can start Faust workers, list agents, models and more,
just by calling app.main()
.
Even if you don’t do that, the faust program is always available and you can point it to any app:
$ faust -A myapp worker -l info
The myapp
argument should point to a Python module/package having
an app
attribute. If the attribute has a different name, please specify
a fully qualified path:
$ faust -A myproj.apps:faust_app worker -l info
Do --help
to get a list of subcommands supported by the app:
$ faust -A myapp --help
To turn your script into the faust command, with the
-A
option already set, add this to the end of the module:
if __name__ == '__main__':
app.main()
If saved as simple.py
you can now execute it as if it was
the faust program:
$ python simple.py worker -l info
Custom CLI Commands¶
To add a custom command to your app, see the examples/simple.py
example in the Faust distribution, where we added a produce
command
used to send example data into the stream processors:
from faust.cli import option
# the full example is in examples/simple.py in the Faust distribution.
# this only shows the command part of this code.
@app.command(
option('--max-latency',
type=float, default=PRODUCE_LATENCY,
help='Add delay of (at most) n seconds between publishing.'),
option('--max-messages',
type=int, default=None,
help='Send at most N messages or 0 for infinity.'),
)
async def produce(self, max_latency: float, max_messages: int):
"""Produce example Withdrawal events."""
num_countries = 5
countries = [f'country_{i}' for i in range(num_countries)]
country_dist = [0.9] + ([0.10 / num_countries] * (num_countries - 1))
num_users = 500
users = [f'user_{i}' for i in range(num_users)]
self.say('Done setting up. SENDING!')
for i in range(max_messages) if max_messages is not None else count():
withdrawal = Withdrawal(
user=random.choice(users),
amount=random.uniform(0, 25_000),
country=random.choices(countries, country_dist)[0],
date=datetime.utcnow().replace(tzinfo=timezone.utc),
)
await withdrawals_topic.send(key=withdrawal.user, value=withdrawal)
if not i % 10000:
self.say(f'+SEND {i}')
if max_latency:
await asyncio.sleep(random.uniform(0, max_latency))
The @app.command
decorator accepts both click.option
and
click.argument
, so you can specify command-line options, as
well as command-line positional arguments.
The daemon
flag can be set to mark the command as a background service
that won’t exit until the user hits Control-c, or the process is
terminated by another signal:
@app.command(
option('--foo', type=float, default=1.33),
daemon=True,
)
async def my_daemon(self, foo: float):
print('STARTING DAEMON')
...
# set up some stuff
# we can return here but the program will not shut down
# until the user hits :kbd:`Control-c`, or the process is terminated
# by signal
return
Command-line Interface¶
Program: faust
¶
The faust umbrella command hosts all command-line functionality
for Faust. Projects may add custom commands using the @app.command
decorator (see CLI Commands).
Options:
-
-A
,
--app
¶
Path of Faust application to use, or the name of a module.
-
--quiet
,
--no-quiet
,
-q
¶
Silence output to <stdout>/<stderr>.
-
--debug
,
--no-debug
¶
Enable debugging output, and the blocking detector.
-
--workdir
,
-W
¶
Working directory to change to after start.
-
--datadir
¶
Directory to keep application state.
-
--json
¶
Return output in machine-readable JSON format.
-
--loop
,
-L
¶
Event loop implementation to use:
aio
(default),gevent
,uvloop
.
Why is examples/word_count.py
used as the program?
The convention for Faust projects is to define an entry point for
the Faust command using app.main()
- see app.main() – Start the faust command-line program.
to see how to do so.
For a standalone program such as examples/word_count.py
this
is accomplished by adding the following at the end of the file:
if __name__ == '__main__':
app.main()
For a project organized in modules (a package) you can add a
package/__main__.py
module:
# package/__main__.py
from package.app import app
app.main()
Or use setuptools entry points so that pip install myproj
installs
a command-line program.
Even if you don’t add an entry point you can always use the faust program by specifying the path to an app.
Either the name of a module having an app
attribute:
$ faust -A examples.word_count
or specifying the attribute directly:
$ faust -A examples.word_count:app
faust --version
- Show version information and exit.¶
Example:
$ python examples/word_count.py --version
word_count.py, version Faust 0.9.39
faust --help
- Show help and exit.¶
Example:
$ python examples/word_count.py --help
Usage: word_count.py [OPTIONS] COMMAND [ARGS]...
Faust command-line interface.
Options:
-L, --loop [aio|gevent|eventlet|uvloop]
Event loop implementation to use.
--json / --no-json Prefer data to be emitted in json format.
-D, --datadir DIRECTORY Directory to keep application state.
-W, --workdir DIRECTORY Working directory to change to after start.
--no-color / --color Enable colors in output.
--debug / --no-debug Enable debugging output, and the blocking
detector.
-q, --quiet / --no-quiet Silence output to <stdout>/<stderr>.
-A, --app TEXT Path of Faust application to use, or the
name of a module.
--version Show the version and exit.
--help Show this message and exit.
Commands:
agents List agents.
model Show model detail.
models List all available models as tabulated list.
reset Delete local table state.
send Send message to agent/topic.
tables List available tables.
worker Start ƒaust worker instance.
faust agents
- List agents defined in this application.¶
Example:
$ python examples/word_count.py agents
┌Agents──────────┬─────────────────────────────────────────────┬──────────────────────────────────────────┐
│ name │ topic │ help │
├────────────────┼─────────────────────────────────────────────┼──────────────────────────────────────────┤
│ @count_words │ word-counts-examples.word_count.count_words │ Count words from blog post article body. │
│ @shuffle_words │ posts │ <N/A> │
└────────────────┴─────────────────────────────────────────────┴──────────────────────────────────────────┘
JSON Output using --json
:
$ python examples/word_count.py --json agents
[{"name": "@count_words",
"topic": "word-counts-examples.word_count.count_words",
"help": "Count words from blog post article body."},
{"name": "@shuffle_words",
"topic": "posts",
"help": "<N/A>"}]
faust models
- List defined serialization models.¶
Example:
$ python examples/word_count.py models
┌Models┬───────┐
│ name │ help │
├──────┼───────┤
│ Word │ <N/A> │
└──────┴───────┘
JSON Output using --json
:
python examples/word_count.py --json models
[{"name": "Word", "help": "<N/A>"}]
faust model <name>
- List model fields by model name.¶
Example:
$ python examples/word_count.py model Word
┌Word───┬──────┬──────────┐
│ field │ type │ default* │
├───────┼──────┼──────────┤
│ word │ str │ * │
└───────┴──────┴──────────┘
JSON Output using --json
:
$ python examples/word_count.py --json model Word
[{"field": "word", "type": "str", "default*": "*"}]
faust reset
- Delete local table state.¶
Warning
This command will result in the destruction of the following files:
- The local database directories/files backing tables
(does not apply if an in-memory store like memory:// is used).
Note
This data is technically recoverable from the Kafka cluster (if intact), but it’ll take a long time to get the data back as you need to consume each changelog topic in total.
It’d be faster to copy the data from any standbys that happen to have the topic partitions you require.
Example:
$ python examples/word_count.py reset
faust send <topic/agent> <message_value>
- Send message.¶
Options:
-
--key-type
,
-K
¶
Name of model to serialize key into.
-
--key-serializer
¶
Override default serializer for key.
-
--value-type
,
-V
¶
Name of model to serialize value into.
-
--value-serializer
¶
Override default serializer for value.
-
--key
,
-k
¶
String value for key (use json if model).
-
--partition
¶
Specific partition to send to.
-
--repeat
,
-r
¶
Send message n times.
-
--min-latency
¶
Minimum delay between sending.
-
--max-latency
¶
Maximum delay between sending.
Examples:
Send to agent by name using @
prefix:
$ python examples/word_count.py send @word_count "foo"
Send to topic by name (no prefix):
$ python examples/word_count.py send mytopic "foo"
{"topic": "mytopic",
"partition": 2,
"topic_partition": ["mytopic", 2],
"offset": 0,
"timestamp": 1520974493620,
"timestamp_type": 0}
To get help:
$ python examples/word_count.py send --help
Usage: word_count.py send [OPTIONS] ENTITY [VALUE]
Send message to agent/topic.
Options:
-K, --key-type TEXT Name of model to serialize key into.
--key-serializer TEXT Override default serializer for key.
-V, --value-type TEXT Name of model to serialize value into.
--value-serializer TEXT Override default serializer for value.
-k, --key TEXT String value for key (use json if model).
--partition INTEGER Specific partition to send to.
-r, --repeat INTEGER Send message n times.
--min-latency FLOAT Minimum delay between sending.
--max-latency FLOAT Maximum delay between sending.
--help Show this message and exit.
faust tables
- List Tables (distributed K/V stores).¶
Example:
$ python examples/word_count.py tables
┌Tables───────┬───────────────────────────────────┐
│ name │ help │
├─────────────┼───────────────────────────────────┤
│ word_counts │ Keep count of words (str to int). │
└─────────────┴───────────────────────────────────┘
JSON Output using --json
:
$ python examples/word_count.py --json tables
[{"name": "word_counts", "help": "Keep count of words (str to int)."}]
faust worker
- Start Faust worker instance.¶
A “worker” starts a single instance of a Faust application.
Options:
-
--logfile
,
-f
¶
Path to logfile (default is <stderr>).
-
--loglevel
,
-l
¶
Logging level to use:
CRIT|ERROR|WARN|INFO|DEBUG
.
-
--blocking-timeout
¶
Blocking detector timeout (requires –debug).
-
--without-web
¶
Do not start embedded web server.
-
--web-host
,
-h
¶
Canonical host name for the web server.
-
--web-port
,
-p
¶
Port to run web server on (default is 6066).
-
--web-bind
,
-b
¶
Network mask to bind web server to (default is “0.0.0.0” - all interfaces).
-
--console-port
¶
When
faust --debug
is enabled this specifies the port to run the aiomonitor console on (default is 50101).
Examples:
$ python examples/word_count.py worker
┌ƒaµS† v1.0.0──────────────────────────────────────────┐
│ id │ word-counts │
│ transport │ kafka://localhost:9092 │
│ store │ rocksdb: │
│ web │ http://localhost:6066/ │
│ log │ -stderr- (warn) │
│ pid │ 46052 │
│ hostname │ grainstate.local │
│ platform │ CPython 3.6.4 (Darwin x86_64) │
│ drivers │ aiokafka=0.4.0 aiohttp=3.0.8 │
│ datadir │ /opt/devel/faust/word-counts-data │
│ appdir │ /opt/devel/faust/word-counts-data/v1 │
└───────────┴──────────────────────────────────────────┘
starting➢ 😊
To get more logging use -l info (or further -l debug):
$ python examples/word_count.py worker -l info
┌ƒaµS† v1.0.0──────────────────────────────────────────┐
│ id │ word-counts │
│ transport │ kafka://localhost:9092 │
│ store │ rocksdb: │
│ web │ http://localhost:6066/ │
│ log │ -stderr- (info) │
│ pid │ 46034 │
│ hostname │ grainstate.local │
│ platform │ CPython 3.6.4 (Darwin x86_64) │
│ drivers │ aiokafka=0.4.0 aiohttp=3.0.8 │
│ datadir │ /opt/devel/faust/word-counts-data │
│ appdir │ /opt/devel/faust/word-counts-data/v1 │
└───────────┴──────────────────────────────────────────┘
starting^[2018-03-13 13:41:39,269: INFO]: [^Worker]: Starting...
[2018-03-13 13:41:39,275: INFO]: [^-App]: Starting...
[2018-03-13 13:41:39,271: INFO]: [^--Web]: Starting...
[2018-03-13 13:41:39,272: INFO]: [^---ServerThread]: Starting...
[2018-03-13 13:41:39,273: INFO]: [^--Web]: Serving on http://localhost:6066/
[2018-03-13 13:41:39,275: INFO]: [^--Monitor]: Starting...
[2018-03-13 13:41:39,275: INFO]: [^--Producer]: Starting...
[2018-03-13 13:41:39,317: INFO]: [^--Consumer]: Starting...
[2018-03-13 13:41:39,325: INFO]: [^--LeaderAssignor]: Starting...
[2018-03-13 13:41:39,325: INFO]: [^--Producer]: Creating topic word-counts-__assignor-__leader
[2018-03-13 13:41:39,325: INFO]: [^--Producer]: Nodes: [0]
[2018-03-13 13:41:39,668: INFO]: [^--Producer]: Topic word-counts-__assignor-__leader created.
[2018-03-13 13:41:39,669: INFO]: [^--ReplyConsumer]: Starting...
[2018-03-13 13:41:39,669: INFO]: [^--Agent]: Starting...
[2018-03-13 13:41:39,673: INFO]: [^---OneForOneSupervisor]: Starting...
[2018-03-13 13:41:39,673: INFO]: [^---Agent*: examples.word_co[.]shuffle_words]: Starting...
[2018-03-13 13:41:39,673: INFO]: [^--Agent]: Starting...
[2018-03-13 13:41:39,674: INFO]: [^---OneForOneSupervisor]: Starting...
[2018-03-13 13:41:39,674: INFO]: [^---Agent*: examples.word_count.count_words]: Starting...
[2018-03-13 13:41:39,674: INFO]: [^--Conductor]: Starting...
[2018-03-13 13:41:39,674: INFO]: [^--TableManager]: Starting...
[2018-03-13 13:41:39,675: INFO]: [^--Stream: <(*)Topic: posts@0x10497e5f8>]: Starting...
[2018-03-13 13:41:39,675: INFO]: [^--Stream: <(*)Topic: wo...s@0x105f73b38>]: Starting...
[...]
To get help use faust worker --help
:
$ python examples/word_count.py worker --help
Usage: word_count.py worker [OPTIONS]
Start ƒaust worker instance.
Options:
-f, --logfile PATH Path to logfile (default is <stderr>).
-l, --loglevel [crit|error|warn|info|debug]
Logging level to use.
--blocking-timeout FLOAT Blocking detector timeout (requires
--debug).
-p, --web-port RANGE[1-65535] Port to run web server on.
-b, --web-bind TEXT
-h, --web-host TEXT Canonical host name for the web server.
--console-port RANGE[1-65535] (when --debug:) Port to run debugger console
on.
--help Show this message and exit.
Sensors - Monitors and Statistics¶
Basics¶
Sensors record information about events in a Faust application as they happen.
You can define custom sensors to record information that you care about, just add it to the list of application sensors. There’s also a default sensor called the “monitor” that record the runtime of messages and events as they go through the worker, the latency of publishing messages, the latency of committing Kafka offsets, and so on.
The web server uses this monitor to present graphs and statistics about your instance, and there’s also a versions of the monitor available that forwards statistics to StatsD, and Datadog.
Monitor¶
The faust.Monitor
is a built-in sensor that captures information like:
Average message processing time (when all agents have processed a message).
Average event processing time (from an event received by an agent to the event is acked.)
The total number of events processed every second.
The total number of events processed every second listed by topic.
The total number of events processed every second listed by agent.
The total number of records written to tables.
Duration of Kafka topic commit operations (latency).
Duration of producing messages (latency).
You can access the state of the monitor, while the worker is running,
in app.monitor
:
@app.agent(app.topic('topic'))
def mytask(events):
async for event in events:
# emit how many events are being processed every second.
print(app.monitor.events_s)
Monitor API Reference¶
Monitor
¶-
class
faust.
Monitor
[source] -
messages_active
Number of messages currently being processed.
-
messages_received_total
Number of messages processed in total.
-
messages_received_by_topic
Count of messages received by topic
-
messages_s
Number of messages being processed this second.
-
events_active
Number of events currently being processed.
-
events_total
Number of events processed in total.
-
events_s
Number of events being processed this second.
-
events_by_stream
Count of events processed by stream
-
events_by_task
Count of events processed by task
-
events_runtime
Deque of run times used for averages
-
events_runtime_avg
Average event runtime over the last second.
-
tables
Mapping of tables
-
commit_latency
Deque of commit latency values
-
send_latency
Deque of send latency values
-
messages_sent
Number of messages sent in total.
-
messages_sent_by_topic
Number of messages sent by topic.
-
TableState
¶-
class
faust.sensors.
TableState
-
TableState.
table
= None
-
TableState.
keys_retrieved
= 0 Number of times a key has been retrieved from this table.
-
TableState.
keys_updated
= 0 Number of times a key has been created/changed in this table.
-
TableState.
keys_deleted
= 0 Number of times a key has been deleted from this table.
-
Sensor API Reference¶
This reference describes the sensor interface and is useful when you want to build custom sensors.
Methods¶
Message Callbacks¶
-
class
faust.
Sensor
[source] -
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source] Message received by a consumer.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source] All streams finished processing message.
- Return type
None
-
Event Callbacks¶
-
class
faust.
Sensor
[source] -
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source] Message sent to a stream as an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source] Event was acknowledged by stream.
Notes
Acknowledged means a stream finished processing the event, but given that multiple streams may be handling the same event, the message cannot be committed before all streams have processed it. When all streams have acknowledged the event, it will go through
on_message_out()
just before offsets are committed.- Return type
None
-
Table Callbacks¶
-
class
faust.
Sensor
[source] -
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source] Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source] Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source] Key deleted from table.
- Return type
None
-
Operation Callbacks¶
-
class
faust.
Sensor
[source] -
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source] Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source] Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source] About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source] Message successfully sent.
- Return type
None
-
Testing¶
Basics¶
To test an agent when unit testing or functional testing, use the special
Agent.test()
mode to send items to the stream while processing it locally:
app = faust.App('test-example')
class Order(faust.Record, serializer='json'):
account_id: str
product_id: str
amount: int
price: float
orders_topic = app.topic('orders', value_type=Order)
orders_for_account = app.Table('order-count-by-account', default=int)
@app.agent(orders_topic)
async def order(orders):
async for order in orders.group_by(Order.account_id):
orders_for_account[order.account_id] += 1
yield order
Our agent reads a stream of orders and keeps a count of them by account id in a distributed table also partitioned by the account id.
To test this agent we use order.test_context()
:
async def test_order():
# start and stop the agent in this block
async with order.test_context() as agent:
order = Order(account_id='1', product_id='2', amount=1, price=300)
# sent order to the test agents local channel, and wait
# the agent to process it.
await agent.put(order)
# at this point the agent already updated the table
assert orders_for_account[order.account_id] == 1
await agent.put(order)
assert orders_for_account[order.account_id] == 2
async def run_tests():
app.conf.store = 'memory://' # tables must be in-memory
await test_order()
if __name__ == '__main__':
import asyncio
loop = asyncio.get_event_loop()
loop.run_until_complete(run_tests())
For the rest of this guide we’ll be using pytest and pytest-asyncio for our examples. If you’re using a different testing framework you may have to adapt them a bit to work.
Testing with pytest¶
Testing that an agent sends to topic/calls another agent.¶
When unit testing you should mock any dependencies of the agent being tested,
If your agent calls another function: mock that function to verify it was called.
If your agent sends a message to a topic: mock that topic to verify a message was sent.
If your agent calls another agent: mock the other agent to verify it was called.
Here’s an example agent that calls another agent:
import faust
app = faust.App('example-test-agent-call')
@app.agent()
async def foo(stream):
async for value in stream:
await bar.send(value)
yield value
@app.agent()
async def bar(stream):
async for value in stream:
yield value + 'YOLO'
To test these two agents you have to test them in isolation of each other:
first test foo
with bar
mocked, then in a different test do bar
:
import pytest
from unittest.mock import Mock, patch
from example import app, foo, bar
@pytest.fixture()
def test_app(event_loop):
"""passing in event_loop helps avoid 'attached to a different loop' error"""
app.finalize()
app.conf.store = 'memory://'
app.flow_control.resume()
return app
@pytest.mark.asyncio()
async def test_foo(test_app):
with patch(__name__ + '.bar') as mocked_bar:
mocked_bar.send = mock_coro()
async with foo.test_context() as agent:
await agent.put('hey')
mocked_bar.send.assert_called_with('hey')
def mock_coro(return_value=None, **kwargs):
"""Create mock coroutine function."""
async def wrapped(*args, **kwargs):
return return_value
return Mock(wraps=wrapped, **kwargs)
@pytest.mark.asyncio()
async def test_bar(test_app):
async with bar.test_context() as agent:
event = await agent.put('hey')
assert agent.results[event.message.offset] == 'heyYOLO'
Note
The pytest-asyncio extension must be installed to run these tests. If you don’t have it use pip to install it:
$ pip install -U pytest-asyncio
Testing and windowed tables¶
If your table is windowed and you want to verify that the value for a key is
correctly set, use table[k].current(event)
to get the value placed within
the window of the current event:
import faust
import pytest
@pytest.mark.asyncio()
async def test_process_order():
app.conf.store = 'memory://'
async with order.test_context() as agent:
order = Order(account_id='1', product_id='2', amount=1, price=300)
event = await agent.put(order)
# windowed table: we select window relative to the current event
assert orders_for_account['1'].current(event) == 1
# in the window 3 hours ago there were no orders:
assert orders_for_account['1'].delta(3600 * 3, event)
class Order(faust.Record, serializer='json'):
account_id: str
product_id: str
amount: int
price: float
app = faust.App('test-example')
orders_topic = app.topic('orders', value_type=Order)
# order count within the last hour (window is a 1-hour TumblingWindow).
orders_for_account = app.Table(
'order-count-by-account', default=int,
).tumbling(3600).relative_to_stream()
@app.agent(orders_topic)
async def process_order(orders):
async for order in orders.group_by(Order.account_id):
orders_for_account[order.account_id] += 1
yield order
Configuration Reference¶
Required Settings¶
id
¶
- type
str
A string uniquely identifying the app, shared across all instances such that two app instances with the same id are considered to be in the same “group”.
This parameter is required.
The id and Kafka
When using Kafka, the id is used to generate app-local topics, and names for consumer groups.
Commonly Used Settings¶
broker
¶
- type
str
- default
[URL("kafka://localhost:9092")]
Faust needs the URL of a “transport” to send and receive messages.
Currently, the only supported production transport is kafka://
.
This uses the aiokafka client under the hood, for consuming and
producing messages.
You can specify multiple hosts at the same time by separating them using the semi-comma:
kafka://kafka1.example.com:9092;kafka2.example.com:9092
Which in actual code looks like this:
app = faust.App(
'id',
broker='kafka://kafka1.example.com:9092;kafka2.example.com:9092',
)
You can also pass a list of URLs:
app = faust.App(
'id',
broker=['kafka://kafka1.example.com:9092',
'kafka://kafka2.example.com:9092'],
)
kafka://
Alias to
aiokafka://
aiokafka://
The recommended transport using the aiokafka client.
Limitations: None
confluent://
Experimental transport using the confluent-kafka client.
- Limitations: Does not do sticky partition assignment (not
suitable for tables), and do not create any necessary internal topics (you have to create them manually).
broker_credentials
¶
New in version 1.5.
- type
- default
None
Specify the authentication mechanism to use when connecting to the broker.
The default is to not use any authentication.
You can enable SASL authentication via plain text:
app = faust.App(
broker_credentials=faust.SASLCredentials(
username='x',
password='y',
))
Warning
Do not use literal strings when specifying passwords in production, as they can remain visible in stack traces.
Instead the best practice is to get the password from a configuration file, or from the environment:
BROKER_USERNAME = os.environ.get('BROKER_USERNAME')
BROKER_PASSWORD = os.environ.get('BROKER_PASSWORD')
app = faust.App(
broker_credentials=faust.SASLCredentials(
username=BROKER_USERNAME,
password=BROKER_PASSWORD,
))
GSSAPI authentication over plain text:
app = faust.App(
broker_credentials=faust.GSSAPICredentials(
kerberos_service_name='faust',
kerberos_domain_name='example.com',
),
)
GSSAPI authentication over SSL:
import ssl
ssl_context = ssl.create_default_context(
purpose=ssl.Purpose.SERVER_AUTH, cafile='ca.pem')
ssl_context.load_cert_chain('client.cert', keyfile='client.key')
app = faust.App(
broker_credentials=faust.GSSAPICredentials(
kerberos_service_name='faust',
kerberos_domain_name='example.com',
ssl_context=ssl_context,
),
)
Provide an SSL context for the Kafka broker connections.
This allows Faust to use a secure SSL/TLS connection for the Kafka connections and enabling certificate-based authentication.
import ssl
ssl_context = ssl.create_default_context(
purpose=ssl.Purpose.SERVER_AUTH, cafile='ca.pem')
ssl_context.load_cert_chain('client.cert', keyfile='client.key')
app = faust.App(..., broker_credentials=ssl_context)
store
¶
- type
str
- default
URL("memory://")
The backend used for table storage.
Tables are stored in-memory by default, but you should
not use the memory://
store in production.
In production, a persistent table store, such as rocksdb://
is
preferred.
cache
¶
New in version 1.2.
- type
str
- default
URL("memory://")
Optional backend used for Memcached-style caching.
URL can be: redis://host
, rediscluster://host
, or memory://
.
processing_guarantee
¶
New in version 1.5.
- type
str
- default
"at_least_once"
The processing guarantee that should be used.
Possible values are “at_least_once” (default) and “exactly_once”.
Note that if exactly-once processing is enabled consumers are configured with
isolation.level="read_committed"
and producers are configured with
retries=Integer.MAX_VALUE
and enable.idempotence=true
per default.
Note that by default exactly-once processing requires a cluster of at least
three brokers what is the recommended setting for production.
For development you can change this, by adjusting broker setting
transaction.state.log.replication.factor
to the number of brokers you want to use.
autodiscover
¶
- type
Union[bool, Iterable[str], Callable[[], Iterable[str]]]
Enable autodiscovery of agent, task, timer, page and command decorators.
Faust has an API to add different asyncio
services and other user
extensions, such as “Agents”, HTTP web views, command-line commands, and
timers to your Faust workers. These can be defined in any module, so to
discover them at startup, the worker needs to traverse packages looking
for them.
Warning
The autodiscovery functionality uses the Venusian library to
scan wanted packages for @app.agent
, @app.page
,
@app.command
, @app.task
and @app.timer
decorators,
but to do so, it’s required to traverse the package path and import
every module in it.
Importing random modules like this can be dangerous so make sure you follow Python programming best practices. Do not start threads; perform network I/O; do test monkey-patching for mocks or similar, as a side effect of importing a module. If you encounter a case such as this then please find a way to perform your action in a lazy manner.
Warning
If the above warning is something you cannot fix, or if it’s out of your
control, then please set autodiscover=False
and make sure the worker
imports all modules where your decorators are defined.
The value for this argument can be:
bool
If
App(autodiscover=True)
is set, the autodiscovery will scan the package name described in theorigin
attribute.The
origin
attribute is automatically set when you start a worker using the faust command line program, for example:faust -A example.simple worker
The
-A
, option specifies the app, but you can also create a shortcut entry point by callingapp.main()
:if __name__ == '__main__': app.main()
Then you can start the faust program by executing for example
python myscript.py worker --loglevel=INFO
, and it will use the correct application.Sequence[str]
The argument can also be a list of packages to scan:
app = App(..., autodiscover=['proj_orders', 'proj_accounts'])
Callable[[], Sequence[str]]
The argument can also be a function returning a list of packages to scan:
def get_all_packages_to_scan(): return ['proj_orders', 'proj_accounts'] app = App(..., autodiscover=get_all_packages_to_scan)
False)
If everything you need is in a self-contained module, or you import the stuff you need manually, just set
autodiscover
to False and don’t worry about it :-)
Django
When using Django and the DJANGO_SETTINGS_MODULE
environment variable is set, the Faust app will scan all packages found
in the INSTALLED_APPS
setting.
If you’re using Django you can use this to scan for
agents/pages/commands in all packages defined in INSTALLED_APPS
.
Faust will automatically detect that you’re using Django and do the right thing if you do:
app = App(..., autodiscover=True)
It will find agents and other decorators in all of the reusable Django applications. If you want to manually control what packages are traversed, then provide a list:
app = App(..., autodiscover=['package1', 'package2'])
or if you want exactly None
packages to be traversed, then
provide a False:
app = App(.., autodiscover=False)
which is the default, so you can simply omit the argument.
Tip
For manual control over autodiscovery, you can also call the
app.discover()
method manually.
version
¶
- type
- default
1
Version of the app, that when changed will create a new isolated instance of the application. The first version is 1, the second version is 2, and so on.
Source topics will not be affected by a version change.
Faust applications will use two kinds of topics: source topics, and
internally managed topics. The source topics are declared by the producer,
and we do not have the opportunity to modify any configuration settings,
like number of partitions for a source topic; we may only consume from
them. To mark a topic as internal, use: app.topic(..., internal=True)
.
timezone
¶
- type
- default
datetime.timezone.utc
The timezone used for date-related functionality such as cronjobs.
New in version 1.4.
datadir
¶
- type
Union[str, pathlib.Path]
- default
Path(f"{app.conf.id}-data")
- environment
FAUST_DATADIR
,F_DATADIR
The directory in which this instance stores the data used by local tables, etc.
See also
The data directory can also be set using the
faust --datadir
option, from the command-line, so there’s usually no reason to provide a default value when creating the app.
tabledir
¶
- type
Union[str, pathlib.Path]
- default
"tables"
The directory in which this instance stores local table data.
Usually you will want to configure the datadir
setting, but if you
want to store tables separately you can configure this one.
If the path provided is relative (it has no leading slash), then the path will
be considered to be relative to the datadir
setting.
id_format
¶
- type
- default
"{id}-v{self.version}"
The format string used to generate the final id
value by combining
it with the version
parameter.
logging_config
¶
New in version 1.5.0.
Optional dictionary for logging configuration, as supported
by logging.config.dictConfig()
.
loghandlers
¶
- type
List[logging.LogHandler]
- default
[]
Specify a list of custom log handlers to use in worker instances.
origin
¶
- type
- default
None
The reverse path used to find the app, for example if the app is located in:
from myproj.app import app
Then the origin
should be "myproj.app"
.
The faust worker program will try to automatically set the origin, but if you are having problems with auto generated names then you can set origin manually.
Serialization Settings¶
key_serializer
¶
- type
Union[str, Codec]
- default
"raw"
Serializer used for keys by default when no serializer is specified, or a model is not being used.
This can be the name of a serializer/codec, or an actual
faust.serializers.codecs.Codec
instance.
See also
The Codecs section in the model guide – for more information about codecs.
value_serializer
¶
- type
Union[str, Codec]
- default
"json"
Serializer used for values by default when no serializer is specified, or a model is not being used.
This can be string, the name of a serializer/codec, or an actual
faust.serializers.codecs.Codec
instance.
See also
The Codecs section in the model guide – for more information about codecs.
Topic Settings¶
topic_replication_factor
¶
- type
- default
1
The default replication factor for topics created by the application.
Note
Generally this should be the same as the configured replication factor for your Kafka cluster.
topic_partitions
¶
- type
- default
8
Default number of partitions for new topics.
Note
This defines the maximum number of workers we could distribute the workload of the application (also sometimes referred as the sharding factor of the application).
topic_allow_declare
¶
New in version 1.5.
- type
- default
True
This setting disables the creation of internal topics.
Faust will only create topics that it considers to be fully owned and managed, such as intermediate repartition topics, table changelog topics etc.
Some Kafka managers does not allow services to create topics, in that case
you should set this to False
.
Advanced Broker Settings¶
broker_client_id
¶
- type
str
- default
f"faust-{VERSION}"
There is rarely any reason to configure this setting.
The client id is used to identify the software used, and is not usually configured by the user.
broker_request_timeout
¶
New in version 1.4.0.
- type
- default
40.0
(forty seconds)
Kafka client request timeout.
broker_commit_every
¶
- type
- default
10_000
Commit offset every n messages.
See also broker_commit_interval
, which is how frequently
we commit on a timer when there are few messages being received.
broker_commit_livelock_soft_timeout
¶
How long time it takes before we warn that the Kafka commit offset has not advanced (only when processing messages).
broker_heartbeat_interval
¶
New in version 1.0.11.
- type
- default
3.0
(three seconds)
How often we send heartbeats to the broker, and also how often we expect to receive heartbeats from the broker.
If any of these time out, you should increase this setting.
broker_session_timeout
¶
New in version 1.0.11.
- type
- default
60.0
(one minute)
How long to wait for a node to finish rebalancing before the broker will consider it dysfunctional and remove it from the cluster.
Increase this if you experience the cluster being in a state of constantly
rebalancing, but make sure you also increase the
broker_heartbeat_interval
at the same time.
broker_max_poll_records
¶
New in version 1.4.
- type
- default
None
The maximum number of records returned in a single call to poll().
If you find that your application needs more time to process messages
you may want to adjust broker_max_poll_records
to tune the
number of records that must be handled on every loop iteration.
Advanced Consumer Settings¶
consumer_max_fetch_size
¶
New in version 1.4.
- type
- default
4*1024**2
The maximum amount of data per-partition the server will return. This size must be at least as large as the maximum message size.
consumer_auto_offset_reset
¶
New in version 1.5.
- type
string
- default
"earliest"
Where the consumer should start reading messages from when there is no initial offset, or the stored offset no longer exists, e.g. when starting a new consumer for the first time. Options include ‘earliest’, ‘latest’, ‘none’.
ConsumerScheduler
¶
New in version 1.5.
- type
Union[str, Type[SchedulingStrategyT]
- default
faust.transport.utils.DefaultSchedulingStrategy
A strategy which dictates the priority of topics and partitions for incoming records. The default strategy does first round-robin over topics and then round-robin over partitions.
Example using a class:
class MySchedulingStrategy(DefaultSchedulingStrategy):
...
app = App(..., ConsumerScheduler=MySchedulingStrategy)
Example using the string path to a class:
app = App(..., ConsumerScheduler='myproj.MySchedulingStrategy')
Advanced Producer Settings¶
producer_compression_type
¶
- type
string
- default
None
The compression type for all data generated by the producer. Valid values are
gzip, snappy, lz4, or None
.
producer_linger_ms
¶
- type
- default
0
Minimum time to batch before sending out messages from the producer.
Should rarely have to change this.
producer_max_request_size
¶
- type
- default
1000000
Maximum size of a request in bytes in the producer.
Should rarely have to change this.
producer_acks
¶
- type
- default
-1
The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the durability of records that are sent. The following settings are common:
0
: Producer will not wait for any acknowledgment from the server at all. The message will immediately be considered sent. (Not recommended)1
: The broker leader will write the record to its local log but will respond without awaiting full acknowledgment from all followers. In this case should the leader fail immediately after acknowledging the record but before the followers have replicated it then the record will be lost.-1
: The broker leader will wait for the full set of in-sync replicas to acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica remains alive. This is the strongest available guarantee.
producer_request_timeout
¶
New in version 1.4.
- type
- default
1200.0
(20 minutes)
Timeout for producer operations. This is set high by default, as this is also the time when producer batches expire and will no longer be retried.
producer_api_version
¶
New in version 1.5.3.
- type
- default
"auto"
Negotiate producer protocol version.
The default value - “auto” means use the latest version supported by both client and server.
Any other version set means you are requesting a specific version of the protocol.
Example Kafka uses:
Kafka headers support was added in Kafka 0.11, so you can specify
api_version="0.10"
to remove the headers from messages.
producer_partitioner
¶
New in version 1.2.
- type
Callable[[bytes, List[int], List[int]], int]
- default
None
The Kafka producer can be configured with a custom partitioner to change how keys are partitioned when producing to topics.
The default partitioner for Kafka is implemented as follows, and can be used as a template for your own partitioner:
import random
from typing import List
from kafka.partitioner.hashed import murmur2
def partition(key: bytes,
all_partitions: List[int],
available: List[int]) -> int:
"""Default partitioner.
Hashes key to partition using murmur2 hashing (from java client)
If key is None, selects partition randomly from available,
or from all partitions if none are currently available
Arguments:
key: partitioning key
all_partitions: list of all partitions sorted by partition ID.
available: list of available partitions in no particular order
Returns:
int: one of the values from ``all_partitions`` or ``available``.
"""
if key is None:
source = available if available else all_paritions
return random.choice(source)
index: int = murmur2(key)
index &= 0x7fffffff
index %= len(all_partitions)
return all_partitions[index]
Advanced Table Settings¶
table_cleanup_interval
¶
How often we cleanup tables to remove expired entries.
Advanced Stream Settings¶
stream_buffer_maxsize
¶
- type
- default
4096
This setting control back pressure to streams and agents reading from streams.
If set to 4096 (default) this means that an agent can only keep at most 4096 unprocessed items in the stream buffer.
Essentially this will limit the number of messages a stream can “prefetch”.
Higher numbers gives better throughput, but do note that if your agent sends messages or update tables (which sends changelog messages).
This means that if the buffer size is large, the
broker_commit_interval
or broker_commit_every
settings
must be set to commit frequently, avoiding back pressure from building up.
A buffer size of 131_072 may let you process over 30,000 events a second as a baseline, but be careful with a buffer size that large when you also send messages or update tables.
stream_recovery_delay
¶
- type
Union[float, datetime.timedelta]
- default
0.0
Number of seconds to sleep before continuing after rebalance. We wait for a bit to allow for more nodes to join/leave before starting recovery tables and then processing streams. This to minimize the chance of errors rebalancing loops.
Changed in version 1.5.3: Disabled by default.
stream_wait_empty
¶
- type
- default
True
This setting controls whether the worker should wait for the currently processing task in an agent to complete before rebalancing or shutting down.
On rebalance/shut down we clear the stream buffers. Those events will be reprocessed after the rebalance anyway, but we may have already started processing one event in every agent, and if we rebalance we will process that event again.
By default we will wait for the currently active tasks, but if your streams are idempotent you can disable it using this setting.
stream_publish_on_commit
¶
- type
- default
False
If enabled we buffer up sending messages until the
source topic offset related to that processing is committed.
This means when we do commit, we may have buffered up a LOT of messages
so commit needs to happen frequently (make sure to decrease
broker_commit_every
).
Advanced Worker Settings¶
worker_redirect_stdouts
¶
- type
- default
True
Enable to have the worker redirect output to sys.stdout
and
sys.stderr
to the Python logging system.
Enabled by default.
Advanced Web Server Settings¶
web_enabled
¶
New in version 1.2.
- type
- default
True
Enable web server and other web components.
This option can also be set using faust worker --without-web
.
web_transport
¶
New in version 1.2.
- type
- default
URL("tcp://")
The network transport used for the web server.
Default is to use TCP, but this setting also enables you to use Unix domain sockets. To use domain sockets specify an URL including the path to the file you want to create like this:
unix:///tmp/server.sock
This will create a new domain socket available in /tmp/server.sock
.
canonical_url
¶
- type
- default
URL(f"http://{web_host}:{web_port}")
You shouldn’t have to set this manually.
The canonical URL defines how to reach the web server on a running
worker node, and is usually set by combining the faust worker --web-host
and faust worker --web-port
command line arguments, not
by passing it as a keyword argument to App
.
web_host
¶
New in version 1.2.
- type
- default
f"{socket.gethostname()}"
Hostname used to access this web server, used for generating
the canonical_url
setting.
This option is usually set by faust worker --web-host
,
not by passing it as a keyword argument to app
.
web_port
¶
New in version 1.2.
- type
- default
6066
A port number between 1024 and 65535 to use for the web server.
This option is usually set by faust worker --web-port
,
not by passing it as a keyword argument to app
.
web_bind
¶
New in version 1.2.
- type
- default
"0.0.0.0"
The IP network address mask that decides what interfaces the web server will bind to.
By default this will bind to all interfaces.
This option is usually set by faust worker --web-bind
,
not by passing it as a keyword argument to app
.
web_in_thread
¶
New in version 1.5.
- type
- default
False
Run the web server in a separate thread.
Use this if you have a large value for stream_buffer_maxsize
and want the web server to be responsive when the worker is otherwise
busy processing streams.
Note
Running the web server in a separate thread means web views and agents will not share the same event loop.
web_cors_options
¶
New in version 1.5.
- type
Mapping[str, ResourceOptions]
- default
None
Enable Cross-Origin Resource Sharing options for all web views in the internal web server.
This should be specified as a dictionary of
URLs to ResourceOptions
:
app = App(..., web_cors_options={
'http://foo.example.com': ResourceOptions(
allow_credentials=True,
allow_methods='*',
)
})
Individual views may override the CORS options used as
arguments to to @app.page
and blueprint.route
.
Advanced Agent Settings¶
agent_supervisor
¶
- type
- default
An agent may start multiple instances (actors) when
the concurrency setting is higher than one (e.g.
@app.agent(concurrency=2)
).
Multiple instances of the same agent are considered to be in the same supervisor group.
The default supervisor is the mode.OneForOneSupervisor
:
if an instance in the group crashes, we restart that instance only.
These are the supervisors supported:
-
If an instance in the group crashes we restart only that instance.
-
If an instance in the group crashes we restart the whole group.
-
If an instance in the group crashes we stop the whole application, and exit so that the Operating System supervisor can restart us.
mode.ForfeitOneForOneSupervisor
If an instance in the group crashes we give up on that instance and never restart it again (until the program is restarted).
mode.ForfeitOneForAllSupervisor
If an instance in the group crashes we stop all instances in the group and never restarted them again (until the program is restarted).
Agent RPC Settings¶
reply_to
¶
- type
str
- default
str(uuid.uuid4())
The name of the reply topic used by this instance. If not set one will be automatically generated when the app is created.
reply_create_topic
¶
- type
bool
- default
False
Set this to True
if you plan on using the RPC with agents.
This will create the internal topic used for RPC replies on that instance at startup.
Extension Settings¶
Agent
¶
- type
Union[str, Type]
- default
The Agent
class to use for agents, or the fully-qualified
path to one (supported by symbol_by_name()
).
Example using a class:
class MyAgent(faust.Agent):
...
app = App(..., Agent=MyAgent)
Example using the string path to a class:
app = App(..., Agent='myproj.agents.Agent')
Stream
¶
- type
Union[str, Type]
- default
The Stream
class to use for streams, or the fully-qualified
path to one (supported by symbol_by_name()
).
Example using a class:
class MyBaseStream(faust.Stream):
...
app = App(..., Stream=MyBaseStream)
Example using the string path to a class:
app = App(..., Stream='myproj.streams.Stream')
Table
¶
- type
Union[str, Type[TableT]]
- default
The Table
class to use for tables, or the fully-qualified
path to one (supported by symbol_by_name()
).
Example using a class:
class MyBaseTable(faust.Table):
...
app = App(..., Table=MyBaseTable)
Example using the string path to a class:
app = App(..., Table='myproj.tables.Table')
SetTable
¶
- type
Union[str, Type[TableT]]
- default
The SetTable
class to use for table-of-set tables,
or the fully-qualified path to one (supported
by symbol_by_name()
).
Example using a class:
class MySetTable(faust.SetTable):
...
app = App(..., Table=MySetTable)
Example using the string path to a class:
app = App(..., Table='myproj.tables.MySetTable')
TableManager
¶
- type
Union[str, Type[TableManagerT]]
- default
The TableManager
used for managing tables,
or the fully-qualified path to one (supported by
symbol_by_name()
).
Example using a class:
from faust.tables import TableManager
class MyTableManager(TableManager):
...
app = App(..., TableManager=MyTableManager)
Example using the string path to a class:
app = App(..., TableManager='myproj.tables.TableManager')
Serializers
¶
- type
Union[str, Type[RegistryT]]
- default
faust.serializers.Registry
The Registry
class used for
serializing/deserializing messages; or the fully-qualified path
to one (supported by symbol_by_name()
).
Example using a class:
from faust.serialiers import Registry
class MyRegistry(Registry):
...
app = App(..., Serializers=MyRegistry)
Example using the string path to a class:
app = App(..., Serializers='myproj.serializers.Registry')
Worker
¶
- type
Union[str, Type[WorkerT]]
- default
The Worker
class used for starting a worker
for this app; or the fully-qualified path
to one (supported by symbol_by_name()
).
Example using a class:
import faust
class MyWorker(faust.Worker):
...
app = faust.App(..., Worker=Worker)
Example using the string path to a class:
app = faust.App(..., Worker='myproj.workers.Worker')
PartitionAssignor
¶
- type
Union[str, Type[PartitionAssignorT]]
- default
faust.assignor.PartitionAssignor
The PartitionAssignor
class used for assigning
topic partitions to worker instances; or the fully-qualified path
to one (supported by symbol_by_name()
).
Example using a class:
from faust.assignor import PartitionAssignor
class MyPartitionAssignor(PartitionAssignor):
...
app = App(..., PartitionAssignor=PartitionAssignor)
Example using the string path to a class:
app = App(..., Worker='myproj.assignor.PartitionAssignor')
LeaderAssignor
¶
- type
Union[str, Type[LeaderAssignorT]]
- default
faust.assignor.LeaderAssignor
The LeaderAssignor
class used for assigning
a master Faust instance for the app; or the fully-qualified path
to one (supported by symbol_by_name()
).
Example using a class:
from faust.assignor import LeaderAssignor
class MyLeaderAssignor(LeaderAssignor):
...
app = App(..., LeaderAssignor=LeaderAssignor)
Example using the string path to a class:
app = App(..., Worker='myproj.assignor.LeaderAssignor')
Router
¶
- type
Union[str, Type[RouterT]]
- default
The Router
class used for routing requests
to a worker instance having the partition for a specific key (e.g. table key);
or the fully-qualified path to one (supported by
symbol_by_name()
).
Example using a class:
from faust.router import Router
class MyRouter(Router):
...
app = App(..., Router=Router)
Example using the string path to a class:
app = App(..., Router='myproj.routers.Router')
Topic
¶
- type
Union[str, Type[TopicT]]
- default
The Topic
class used for defining new topics; or the
fully-qualified path to one (supported by
symbol_by_name()
).
Example using a class:
import faust
class MyTopic(faust.Topic):
...
app = faust.App(..., Topic=MyTopic)
Example using the string path to a class:
app = faust.App(..., Topic='myproj.topics.Topic')
HttpClient
¶
- type
Union[str, Type[HttpClientT]]
- default
aiohttp.client.ClientSession
The aiohttp.client.ClientSession
class used as a HTTP client; or the
fully-qualified path to one (supported by
symbol_by_name()
).
Example using a class:
import faust
from aiohttp.client import ClientSession
class HttpClient(ClientSession):
...
app = faust.App(..., HttpClient=HttpClient)
Example using the string path to a class:
app = faust.App(..., HttpClient='myproj.http.HttpClient')
Monitor
¶
- type
Union[str, Type[SensorT]]
- default
The Monitor
class as the main sensor
gathering statistics for the application; or the
fully-qualified path to one (supported by
symbol_by_name()
).
Example using a class:
import faust
from faust.sensors import Monitor
class MyMonitor(Monitor):
...
app = faust.App(..., Monitor=MyMonitor)
Example using the string path to a class:
app = faust.App(..., Monitor='myproj.monitors.Monitor')
Installation¶
Installation¶
You can install Faust either via the Python Package Index (PyPI) or from source.
To install using pip:
$ pip install -U faust
Bundles¶
Faust also defines a group of setuptools extensions that can be used to install Faust and the dependencies for a given feature.
You can specify these in your requirements or on the pip command-line by using brackets. Separate multiple bundles using the comma:
$ pip install "faust[rocksdb]"
$ pip install "faust[rocksdb,uvloop,fast,redis]"
The following bundles are available:
faust[redis]
for using Redis_ as a simple caching backend (Memcached-style).
faust[fast]
for installing all the available C speedup extensions to Faust core.
faust[datadog]
for using the Datadog Faust monitor.
faust[statsd]
for using the Statsd Faust monitor.
faust[debug]
for using aiomonitor to connect and debug a running Faust worker.
faust[setproctitle]
when the setproctitle module is installed the Faust worker will use it to set a nicer process name in ps/top listings. Also installed with the
fast
anddebug
bundles.
Downloading and installing from source¶
Download the latest version of Faust from http://pypi.org/project/faust
You can install it by doing:
$ tar xvfz faust-0.0.0.tar.gz
$ cd faust-0.0.0
$ python setup.py build
# python setup.py install
The last command must be executed as a privileged user if you are not currently using a virtualenv.
Kafka - The basics you need to know¶
Kafka is a distributed streaming platform which uses logs as the unit of storage for messages passed within the system. It is horizontally scalable, fault-tolerant, fast, and runs in production in thousands of companies. Likely your business is already using it in some form.
What you must know about Apache Kafka to use Faust¶
Topics
A topic is a stream name to which records are published. Topics in Kafka are always multi-subscriber; that is, a topic can have zero, one, or many consumers that subscribe to the data written to it. Topics are also the base abstraction of where data lives within Kafka. Each topic is backed by logs which are partitioned and distributed.
Faust uses the abstraction of a topic to both consume data from a stream as well as publish data to more streams represented by Kafka topics. A Faust application needs to be consuming from at least one topic and may create many intermediate topics as a by-product of your streaming application. Every topic that is not created by Faust internally can be thought of as a source (for your application to process) or sink (for other systems to pick up).
Partitions
Partitions are the fundamental unit within Kafka where data lives. Every topic is split into one or more partitions. These partitions are represented internally as logs and messages always make their way to one partition (log). Each partition is replicated and has one leader at any point in time.
Faust uses the notion of a partition to maintain order amongst messages and as a way to split the processing of data to increase throughput. A Faust application uses the notion of a “key” to make sure that messages that should appear together and be processed on the same box do end up on the same box.
Fault Tolerance
Every partition is replicated with the total copies represented by the In Sync Replicas (ISR). Every ISR is a candidate to take over as the new leader should the current leader fail. The maximum number of faulty Kafka brokers that can be tolerated is the number of ISR - 1. I.e., if every partition has three replicas, all fault tolerance guarantees hold as long as at least one replica is functional
Faust has the same guarantees that Kafka offers with regards to fault tolerance of the data.
Distribution of load/work
For every partition, all reads and writes are routed to the leader of that partition. For a specific topic, the load is as distributed as the number of partitions. Note: Since the partition is the lowest degree of parallel processing of messages, the number of partitions control how much many parallel instances of the consumers can operate on messages.
Faust uses parallel consumers and therefore is also limited by the number of partitions to dictate how many concurrent Faust application instances can run to distribute work. Extra Faust application instances beyond the source topic partition count will be idle and not improve message processing rates.
- Offsets
For every <topic, partition> combination Kafka keeps track of the offset of messages in the log to know where new messages should be appended. On a consumer level, offsets are maintained at the <group, topic, partition> level for consumers to know where to continue consuming for a given “group”. The group acts as a namespace for consumers to register when multiple consumers want to share the load on a single topic.
Kafka maintains processing guarantees of at least once by committing offsets after message consumption. Once an offset has been committed at the consumer level, the message at that offset for the <group, topic, partition> will not be reread.
Faust uses the notion of a group to maintain a namespace within an app. Faust commits offsets after when a message is processed through all of its operations. Faust allows a configurable commit interval which makes sure that all messages that have been processed completely since the last interval will be committed.
Log Compaction
Log compaction is a methodology Kafka uses to make sure that as data for a key changes it will not affect the size of the log such that every state change is maintained for all time. Only the most recent value is guaranteed to be available. Periodic compaction removes all values for a key except the last one.
Tables in Faust use log compaction to ensure table state can be recovered without a large space overhead.
This summary and information about Kafka is adapted from original documentation on Kafka available at https://kafka.apache.org/
Debugging¶
Debugging with aiomonitor¶
To use the debugging console you first need to install the aiomonitor library:
$ pip install aiomonitor
You can also install it as part of a bundle:
$ pip install -U faust[debug]
After aiomonitor is installed you may start the worker with the
--debug
option enabled:
$ faust -A myapp --debug worker -l info
┌ƒaµS† v0.9.20─────────────────────────────────────────┐
│ id │ word-counts │
│ transport │ kafka://localhost:9092 │
│ store │ rocksdb: │
│ web │ http://localhost:6066/ │
│ log │ -stderr- (info) │
│ pid │ 55522 │
│ hostname │ grainstate.local │
│ platform │ CPython 3.6.3 (Darwin x86_64) │
│ drivers │ aiokafka=0.3.2 aiohttp=2.3.7 │
│ datadir │ /opt/devel/faust/word-counts-data │
└───────────┴──────────────────────────────────────────┘
[2018-01-04 12:41:07,635: INFO]: Starting aiomonitor at 127.0.0.1:50101
[2018-01-04 12:41:07,637: INFO]: Starting console at 127.0.0.1:50101
[2018-01-04 12:41:07,638: INFO]: [^Worker]: Starting...
[2018-03-13 13:41:39,275: INFO]: [^-App]: Starting...
[2018-01-04 12:41:07,638: INFO]: [^--Web]: Starting...
[...]
From the log output you can tell that the aiomonitor console was
started on the local port 50101. If you get a different output,
such as that the port is already taken you can set a custom port
using the --console-port
.
Once you have the port number, you can telnet into the console to use it:
$ telnet localhost 50101
Trying 127.0.0.1...
Connected to localhost.
Escape character is '^]'.
Asyncio Monitor: 38 tasks running
Type help for commands
monitor >>>
Type help
and then press enter to see a list of available commands:
monitor >>> help
Commands:
ps : Show task table
where taskid : Show stack frames for a task
cancel taskid : Cancel an indicated task
signal signame : Send a Unix signal
console : Switch to async Python REPL
quit : Leave the monitor
monitor >>>
To exit out of the console you can either type quit at the monitor >>
prompt. If that is unresponsive you may hit the special telnet escape character
(Ctrl-]), to drop you into the telnet command console, and from
there you just type quit to exit out of the telnet session:
$> telnet localhost 50101
Trying 127.0.0.1...
Connected to localhost.
Escape character is '^]'.
Asyncio Monitor: 38 tasks running
Type help for commands
monitor >>> ^]
telnet> quit
Connection closed.
Workers Guide¶
Starting the worker¶
If you have defined a Faust app in the module proj.py
:
# proj.py
import faust
app = faust.App('proj', broker='kafka://localhost:9092')
@app.agent()
async def process(stream):
async for value in stream:
print(value)
You can start the worker in the foreground by executing the command:
$ faust -A proj worker -l info
For a full list of available command-line options simply do:
$ faust worker --help
You can start multiple workers for the same app on the same machine, but be sure to provide a unique web server port to each worker, and also a unique data directory.
Start first worker:
$ faust --datadir=/var/faust/worker1 -A proj -l info worker --web-port=6066
Then start the second worker:
$ faust --datadir=/var/faust/worker2 -A proj -l info worker --web-port=6067
Stopping the worker¶
Shutdown should be accomplished using the TERM
signal.
When shutdown is initiated the worker will finish all currently executing
tasks before it actually terminates. If these tasks are important, you should
wait for it to finish before doing anything drastic, like sending the KILL
signal.
If the worker won’t shutdown after considerate time, for being
stuck in an infinite-loop or similar, you can use the KILL
signal to
force terminate the worker. The tasks that did not complete will be executed
again by another worker.
Starting subprocesses
For Faust applications that start subprocesses as a side
effect of processing the stream, you should know that the “double-fork”
problem on Unix means that the worker will not be able to reap its children
when killed using the KILL
signal.
To kill the worker and any child processes, this command usually does the trick:
$ pkill -9 -f 'faust'
If you don’t have the pkill command on your system, you can use the slightly longer version:
$ ps auxww | grep 'faust' | awk '{print $2}' | xargs kill -9
Restarting the worker¶
To restart the worker you should send the TERM signal and start a new instance.
Process Signals¶
The worker’s main process overrides the following signals:
|
Warm shutdown, wait for tasks to complete. |
|
Cold shutdown, terminate ASAP |
|
Dump traceback for all active threads. |
Frequently Asked Questions (FAQ)¶
FAQ¶
Can I use Faust with Django/Flask/etc.?¶
Yes! Use gevent or eventlet as a bridge to integrate with
asyncio
.
Using gevent¶
This approach works with any blocking Python library that can work with gevent.
Using gevent requires you to install the aiogevent module, and you can install this as a bundle with Faust:
$ pip install -U faust[gevent]
Then to actually use gevent as the event loop you have to either
use the -L
option to the faust program:
$ faust -L gevent -A myproj worker -l info
or add import mode.loop.gevent
at the top of your entry point script:
#!/usr/bin/env python3
import mode.loop.gevent
REMEMBER: It’s very important that this is at the very top of the module, and that it executes before you import libraries.
Using eventlet¶
This approach works with any blocking Python library that can work with eventlet.
Using eventlet requires you to install the aioeventlet module, and you can install this as a bundle along with Faust:
$ pip install -U faust[eventlet]
Then to actually use eventlet as the event loop you have to either
use the -L
argument to the faust program:
$ faust -L eventlet -A myproj worker -l info
or add import mode.loop.eventlet
at the top of your entry point script:
#!/usr/bin/env python3
import mode.loop.eventlet # noqa
Warning
It’s very important this is at the very top of the module, and that it executes before you import libraries.
Can I use Faust with Tornado?¶
Yes! Use the tornado.platform.asyncio
bridge:
http://www.tornadoweb.org/en/stable/asyncio.html
Can I use Faust with Twisted?¶
Yes! Use the asyncio
reactor implementation:
https://twistedmatrix.com/documents/17.1.0/api/twisted.internet.asyncioreactor.html
Will you support Python 3.5 or earlier?¶
There are no immediate plans to support Python 3.5, but you are welcome to contribute to the project.
Here are some of the steps required to accomplish this:
Source code transformation to rewrite variable annotations to comments
for example, the code:
class Point: x: int = 0 y: int = 0 must be rewritten into:: class Point: x = 0 # type: int y = 0 # type: int
Source code transformation to rewrite async functions
for example, the code:
async def foo(): await asyncio.sleep(1.0)
must be rewritten into:
@coroutine def foo(): yield from asyncio.sleep(1.0)
Will you support Python 2?¶
There are no plans to support Python 2, but you are welcome to contribute to the project (details in the question above is relevant also for Python 2).
I get a maximum number of open files exceeded error by RocksDB when running a Faust app locally. How can I fix this?¶
You may need to increase the limit for the maximum number of open files. The following post explains how to do so on OS X: https://blog.dekstroza.io/ulimit-shenanigans-on-osx-el-capitan/
What kafka versions faust supports?¶
Faust supports kafka with version >= 0.10.
API Reference¶
- Release
1.5
- Date
Apr 17, 2019
Faust¶
faust
¶
Python Stream processing.
-
class
faust.
Agent
(fun: Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, isolated_partitions: bool = False, use_reply_headers: bool = None, **kwargs) → None[source]¶ Agent.
This is the type of object returned by the
@app.agent
decorator.-
supervisor
= None¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
clone
(*, cls: Type[faust.types.agents.AgentT] = None, **kwargs) → faust.types.agents.AgentT[source]¶ - Return type
AgentT
[]
-
test_context
(channel: faust.types.channels.ChannelT = None, supervisor_strategy: mode.types.supervisors.SupervisorStrategyT = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, **kwargs) → faust.types.agents.AgentTestWrapperT[source]¶ - Return type
-
actor_from_stream
(stream: Optional[faust.types.streams.StreamT], *, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, channel: faust.types.channels.ChannelT = None) → faust.types.agents.ActorT[Union[AsyncIterable, Awaitable]][source]¶ - Return type
ActorT
[]
-
add_sink
(sink: Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]) → None[source]¶ - Return type
None
-
stream
(channel: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
coroutine
ask
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]¶ - Return type
-
coroutine
ask_nowait
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]¶ - Return type
-
coroutine
cast
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → None[source]¶ - Return type
None
-
coroutine
join
(self, values: Union[AsyncIterable[Union[bytes, faust.types.core._ModelT, Any]], Iterable[Union[bytes, faust.types.core._ModelT, Any]]], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvjoin
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
kvmap
(items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]¶ - Return type
-
logger
= <Logger faust.agents.agent (WARNING)>¶
-
map
(values: Union[AsyncIterable, Iterable], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[source]¶ - Return type
AsyncIterator
[+T_co]
-
coroutine
on_isolated_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_isolated_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
- Return type
None
- Return type
None
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to topic used by agent.
- Return type
-
channel_iterator
¶ - Return type
AsyncIterator
[+T_co]
-
-
class
faust.
App
(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = None, beacon: mode.utils.types.trees.NodeT = None, **options) → None[source]¶ Faust Application.
- Parameters
id (str) – Application ID.
- Keyword Arguments
loop (asyncio.AbstractEventLoop) – optional event loop to use.
See also
Application Parameters – for supported keyword arguments.
-
class
BootStrategy
(app: faust.types.app.AppT, *, enable_web: bool = None, enable_kafka: bool = None, enable_kafka_producer: bool = None, enable_kafka_consumer: bool = None, enable_sensors: bool = None) → None¶ App startup strategy.
The startup strategy defines the graph of services to start when the Faust worker for an app starts.
-
enable_kafka
= True¶
-
enable_kafka_consumer
= None¶
-
enable_kafka_producer
= None¶
-
enable_sensors
= True¶
-
enable_web
= None¶
-
-
class
Settings
(id: str, *, version: int = None, broker: Union[str, yarl.URL, List[yarl.URL]] = None, broker_client_id: str = None, broker_request_timeout: Union[datetime.timedelta, float, str] = None, broker_credentials: Union[faust.types.auth.CredentialsT, ssl.SSLContext] = None, broker_commit_every: int = None, broker_commit_interval: Union[datetime.timedelta, float, str] = None, broker_commit_livelock_soft_timeout: Union[datetime.timedelta, float, str] = None, broker_session_timeout: Union[datetime.timedelta, float, str] = None, broker_heartbeat_interval: Union[datetime.timedelta, float, str] = None, broker_check_crcs: bool = None, broker_max_poll_records: int = None, agent_supervisor: Union[_T, str] = None, store: Union[str, yarl.URL] = None, cache: Union[str, yarl.URL] = None, web: Union[str, yarl.URL] = None, web_enabled: bool = True, processing_guarantee: Union[str, faust.types.enums.ProcessingGuarantee] = None, timezone: datetime.tzinfo = None, autodiscover: Union[bool, Iterable[str], Callable[Iterable[str]]] = None, origin: str = None, canonical_url: Union[str, yarl.URL] = None, datadir: Union[pathlib.Path, str] = None, tabledir: Union[pathlib.Path, str] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, logging_config: Dict = None, loghandlers: List[logging.Handler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = None, topic_allow_declare: bool = None, id_format: str = None, reply_to: str = None, reply_to_prefix: str = None, reply_create_topic: bool = None, reply_expires: Union[datetime.timedelta, float, str] = None, ssl_context: ssl.SSLContext = None, stream_buffer_maxsize: int = None, stream_wait_empty: bool = None, stream_ack_cancelled_tasks: bool = None, stream_ack_exceptions: bool = None, stream_publish_on_commit: bool = None, stream_recovery_delay: Union[datetime.timedelta, float, str] = None, producer_linger_ms: int = None, producer_max_batch_size: int = None, producer_acks: int = None, producer_max_request_size: int = None, producer_compression_type: str = None, producer_partitioner: Union[_T, str] = None, producer_request_timeout: Union[datetime.timedelta, float, str] = None, producer_api_version: str = None, consumer_max_fetch_size: int = None, consumer_auto_offset_reset: str = None, web_bind: str = None, web_port: int = None, web_host: str = None, web_transport: Union[str, yarl.URL] = None, web_in_thread: bool = None, web_cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[_T, str] = None, ConsumerScheduler: Union[_T, str] = None, Stream: Union[_T, str] = None, Table: Union[_T, str] = None, SetTable: Union[_T, str] = None, TableManager: Union[_T, str] = None, Serializers: Union[_T, str] = None, Worker: Union[_T, str] = None, PartitionAssignor: Union[_T, str] = None, LeaderAssignor: Union[_T, str] = None, Router: Union[_T, str] = None, Topic: Union[_T, str] = None, HttpClient: Union[_T, str] = None, Monitor: Union[_T, str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None¶ -
-
LeaderAssignor
¶ - Return type
-
PartitionAssignor
¶ - Return type
-
TableManager
¶ - Return type
-
agent_supervisor
¶ - Return type
-
autodiscover
= False¶
-
broker_check_crcs
= True¶
-
broker_client_id
= 'faust-1.5.5'¶
-
broker_commit_every
= 10000¶
-
broker_credentials
¶ - Return type
-
consumer_auto_offset_reset
= 'earliest'¶
-
consumer_max_fetch_size
= 4194304¶
-
id_format
= '{id}-v{self.version}'¶
-
key_serializer
= 'raw'¶
-
logging_config
= None¶
-
processing_guarantee
¶ - Return type
-
producer_acks
= -1¶
-
producer_api_version
= 'auto'¶
-
producer_compression_type
= None¶
-
producer_linger_ms
= 0¶
-
producer_max_batch_size
= 16384¶
-
producer_max_request_size
= 1000000¶
-
producer_partitioner
¶
-
reply_create_topic
= False¶
-
reply_to_prefix
= 'f-reply-'¶
-
ssl_context
= None¶
-
stream_ack_cancelled_tasks
= True¶
-
stream_ack_exceptions
= True¶
-
stream_buffer_maxsize
= 4096¶
-
stream_publish_on_commit
= False¶
-
stream_wait_empty
= True¶
-
table_standby_replicas
= 1¶
-
timezone
= datetime.timezone.utc¶
-
topic_allow_declare
= True¶
-
topic_partitions
= 8¶
-
topic_replication_factor
= 1¶
-
value_serializer
= 'json'¶
-
web_bind
= '0.0.0.0'¶
-
web_cors_options
= None¶
-
web_host
= 'build-8929922-project-230058-faust'¶
-
web_in_thread
= False¶
-
web_port
= 6066¶
-
worker_redirect_stdouts
= True¶
-
worker_redirect_stdouts_level
= 'WARN'¶
-
-
client_only
= False¶ Set this to True if app should only start the services required to operate as an RPC client (producer and simple reply consumer).
-
producer_only
= False¶ Set this to True if app should run without consumer/tables.
-
tracer
= None¶ Optional tracing support.
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
config_from_object
(obj: Any, *, silent: bool = False, force: bool = False) → None[source]¶ Read configuration from object.
Object is either an actual object or the name of a module to import.
Examples
>>> app.config_from_object('myproj.faustconfig')
>>> from myproj import faustconfig >>> app.config_from_object(faustconfig)
-
discover
(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[Any] = [<built-in method search of _sre.SRE_Pattern object>, '.__main__']) → None[source]¶ - Return type
None
-
topic
(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, maxsize: int = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → faust.types.topics.TopicT[source]¶ Create topic description.
Topics are named channels (for example a Kafka topic), that exist on a server. To make an ephemeral local communication channel use:
channel()
.See also
- Return type
TopicT
[]
-
channel
(*, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, maxsize: int = None, loop: asyncio.events.AbstractEventLoop = None) → faust.types.channels.ChannelT[source]¶ Create new channel.
By default this will create an in-memory channel used for intra-process communication, but in practice channels can be backed by any transport (network or even means of inter-process communication).
See also
- Return type
ChannelT
[]
-
agent
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT][source]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
actor
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
task
(fun: Union[Callable[AppT, Awaitable], Callable[Awaitable]] = None, *, on_leader: bool = False, traced: bool = True) → Union[Callable[Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]], Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]]], Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]][source]¶ Define an async def function to be started with the app.
This is like
timer()
but a one-shot task only executed at worker startup (after recovery and the worker is fully ready for operation).The function may take zero, or one argument. If the target function takes an argument, the
app
argument is passed:>>> @app.task >>> async def on_startup(app): ... print('STARTING UP: %r' % (app,))
Nullary functions are also supported:
>>> @app.task >>> async def on_startup(): ... print('STARTING UP')
-
timer
(interval: Union[datetime.timedelta, float, str], on_leader: bool = False, traced: bool = True, name: str = None, max_drift_correction: float = 0.1) → Callable[source]¶ Define an async def function to be run at periodic intervals.
Like
task()
, but executes periodically until the worker is shut down.This decorator takes an async function and adds it to a list of timers started with the app.
- Parameters
interval (Seconds) – How often the timer executes in seconds.
on_leader (bool) – Should the timer only run on the leader?
Example
>>> @app.timer(interval=10.0) >>> async def every_10_seconds(): ... print('TEN SECONDS JUST PASSED')
>>> app.timer(interval=5.0, on_leader=True) >>> async def every_5_seconds(): ... print('FIVE SECONDS JUST PASSED. ALSO, I AM THE LEADER!')
- Return type
-
crontab
(cron_format: str, *, timezone: datetime.tzinfo = None, on_leader: bool = False, traced: bool = True) → Callable[source]¶ Define periodic task using Crontab description.
This is an
async def
function to be run at the fixed times, defined by the Cron format.Like
timer()
, but executes at fixed times instead of executing at certain intervals.This decorator takes an async function and adds it to a list of Cronjobs started with the app.
- Parameters
cron_format (
str
) – The Cron spec defining fixed times to run the decorated function.- Keyword Arguments
timezone – The timezone to be taken into account for the Cron jobs. If not set value from
timezone
will be taken.on_leader – Should the Cron job only run on the leader?
Example
>>> @app.crontab(cron_format='30 18 * * *', timezone=pytz.timezone('US/Pacific')) >>> async def every_6_30_pm_pacific(): ... print('IT IS 6:30pm')
>>> app.crontab(cron_format='30 18 * * *', on_leader=True) >>> async def every_6_30_pm(): ... print('6:30pm UTC; ALSO, I AM THE LEADER!')
- Return type
-
service
(cls: Type[mode.types.services.ServiceT]) → Type[mode.types.services.ServiceT][source]¶ Decorate
mode.Service
to be started with the app.Examples
from mode import Service @app.service class Foo(Service): ...
-
stream
(channel: Union[AsyncIterable, Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]¶ Create new stream from channel/topic/iterable/async iterable.
- Parameters
channel (
Union
[AsyncIterable
[+T_co],Iterable
[+T_co]]) – Iterable to stream over (async or non-async).
- Return type
StreamT
[+T_co]- Returns
to iterate over events in the stream.
- Return type
-
Table
(name: str, *, default: Callable[Any] = None, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ Define new table.
- Parameters
name (
str
) – Name used for table, note that two tables living in the same application cannot have the same name.default (
Optional
[Callable
[[],Any
]]) – A callable, or type that will return a default value for keys missing in this table.window (
Optional
[WindowT
]) – A windowing strategy to wrap this window in.
Examples
>>> table = app.Table('user_to_amount', default=int) >>> table['George'] 0 >>> table['Elaine'] += 1 >>> table['Elaine'] += 1 >>> table['Elaine'] 2
- Return type
TableT
[~KT, ~VT]
-
SetTable
(name: str, *, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ - Return type
TableT
[~KT, ~VT]
-
page
(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, name: str = None) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Type[faust.web.views.View]][source]¶
-
table_route
(table: faust.types.tables.CollectionT, shard_param: str = None, *, query_param: str = None, match_info: str = None) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
command
(*options, base: Optional[Type[faust.app.base._AppCommand]] = None, **kwargs) → Callable[Callable, Type[faust.app.base._AppCommand]][source]¶
-
trace
(name: str, trace_enabled: bool = True, **extra_context) → ContextManager[source]¶ - Return type
ContextManager
[+T_co]
-
traced
(fun: Callable, name: str = None, sample_rate: float = 1.0, **context) → Callable[source]¶ - Return type
-
FlowControlQueue
(maxsize: int = None, *, clear_on_resume: bool = False, loop: asyncio.events.AbstractEventLoop = None) → mode.utils.queues.ThrowableQueue[source]¶ Like
asyncio.Queue
, but can be suspended/resumed.- Return type
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ Commit offset for acked messages in specified topics’.
Warning
This will commit acked messages in all topics if the topics argument is passed in as
None
.- Return type
-
logger
= <Logger faust.app.base (WARNING)>¶
-
coroutine
maybe_start_client
(self) → None[source]¶ Start the app in Client-Only mode if not started as Server.
- Return type
None
-
coroutine
on_first_start
(self) → None[source]¶ Service started for the first time in this process.
- Return type
None
-
coroutine
on_init_extra_service
(self, service: Union[mode.types.services.ServiceT, Type[mode.types.services.ServiceT]]) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
coroutine
send
(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send event to channel/topic.
- Parameters
channel (
Union
[ChannelT
[],str
]) – Channel/topic or the name of a topic to send event to.partition (
Optional
[int
]) – Specific partition to send to. If not set the partition will be chosen by the partitioner.timestamp (
Optional
[float
]) – Epoch seconds (from Jan 1 1970 UTC) to use as the message timestamp. Defaults to current time.headers (
Union
[List
[Tuple
[str
,bytes
]],Mapping
[str
,bytes
],None
]) – Mapping of key/value pairs, or iterable of key value pairs to use as headers for the message.key_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).value_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).callback (
Optional
[Callable
[[FutureMessage
[]],Union
[None
,Awaitable
[None
]]]]) –Called after the message is fully delivered to the channel, but not to the consumer. Signature must be unary as the
FutureMessage
future is passed to it.The resulting
faust.types.tuples.RecordMetadata
object is then available asfut.result()
.
- Return type
-
coroutine
start_client
(self) → None[source]¶ Start the app in Client-Only mode necessary for RPC requests.
Notes
Once started as a client the app cannot be restarted as Server.
- Return type
None
-
transport
¶ Message transport. :rtype:
TransportT
-
cache
¶ - Return type
-
topics
[source]¶ Topic Conductor.
This is the mediator that moves messages fetched by the Consumer into the streams.
It’s also a set of registered topics by string topic name, so you can check if a topic is being consumed from by doing
topic in app.topics
.
-
flow_control
[source]¶ Flow control of streams.
This object controls flow into stream queues, and can also clear all buffers.
-
http_client
¶ HTTP Client Session. :rtype:
ClientSession
-
class
faust.
Channel
(app: faust.types.app.AppT, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, is_iterator: bool = False, queue: mode.utils.queues.ThrowableQueue = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Create new channel.
- Parameters
app (
AppT
[]) – The app that created this channel (app.channel()
)key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – The Model used for keys in this channel.value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – The Model used for values in this channel.maxsize (
Optional
[int
]) – The maximum number of messages this channel can hold. If exceeded any newput
call will block until a message is removed from the channel.loop (
Optional
[AbstractEventLoop
]) – Theasyncio
event loop to use.
-
queue
¶ - Return type
-
clone
(*, is_iterator: bool = None, **kwargs) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
clone_using_queue
(queue: asyncio.queues.Queue) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
stream
(**kwargs) → faust.types.streams.StreamT[source]¶ Create stream reading from this channel.
- Return type
StreamT
[+T_co]
-
as_future_message
(key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → faust.types.tuples.FutureMessage[source]¶ - Return type
-
prepare_headers
(headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None]) → Union[List[Tuple[str, bytes]], MutableMapping[str, bytes]][source]¶
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
prepare_value
(value: Union[bytes, faust.types.core._ModelT, Any], value_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]¶ - Return type
EventT
[]
-
coroutine
get
(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]¶ - Return type
-
coroutine
on_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_key_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_value_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to channel.
- Return type
-
class
faust.
ChannelT
(app: faust.types.channels._AppT, *, key_type: faust.types.channels._ModelArg = None, value_type: faust.types.channels._ModelArg = None, is_iterator: bool = False, queue: mode.utils.queues.ThrowableQueue = None, maxsize: int = None, root: Optional[faust.types.channels.ChannelT] = None, active_partitions: Set[faust.types.tuples.TP] = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
clone
(*, is_iterator: bool = None, **kwargs) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
clone_using_queue
(queue: asyncio.queues.Queue) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
as_future_message
(key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → faust.types.tuples.FutureMessage[source]¶ - Return type
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
prepare_value
(value: Union[bytes, faust.types.core._ModelT, Any], value_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
queue
¶ - Return type
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.channels._EventT[source]¶ - Return type
_EventT
-
coroutine
get
(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]¶ - Return type
-
coroutine
on_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_key_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_value_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.
Event
(app: faust.types.app.AppT, key: Union[bytes, faust.types.core._ModelT, Any, None], value: Union[bytes, faust.types.core._ModelT, Any], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], message: faust.types.tuples.Message) → None[source]¶ An event received on a channel.
Notes
Events have a key and a value:
event.key, event.value
They also have a reference to the original message (if available), such as a Kafka record:
event.message.offset
Iterating over channels/topics yields Event:
- async for event in channel:
…
Iterating over a stream (that in turn iterate over channel) yields Event.value:
async for value in channel.stream() # value is event.value ...
If you only have a Stream object, you can also access underlying events by using
Stream.events
.For example:
async for event in channel.stream.events(): ...
Also commonly used for finding the “current event” related to a value in the stream:
stream = channel.stream() async for event in stream.events(): event = stream.current_event message = event.message topic = event.message.topic
You can retrieve the current event in a stream to:
Get access to the serialized key+value.
Get access to message properties like, what topic+partition the value was received on, or its offset.
If you want access to both key and value, you should use
stream.items()
instead.async for key, value in stream.items(): ...
stream.current_event
can also be accessed but you must take extreme care you are using the correct stream object. Methods such as.group_by(key)
and.through(topic)
returns cloned stream objects, so in the example:The best way to access the current_event in an agent is to use the
ContextVar
:from faust import current_event @app.agent(topic) async def process(stream): async for value in stream: event = current_event()
-
coroutine
forward
(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = <object object>, value: Union[bytes, faust.types.core._ModelT, Any] = <object object>, partition: int = None, timestamp: float = None, headers: Any = <object object>, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Forward original message (will not be reserialized).
- Return type
-
coroutine
send
(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = <object object>, value: Union[bytes, faust.types.core._ModelT, Any] = <object object>, partition: int = None, timestamp: float = None, headers: Any = <object object>, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send object to channel.
- Return type
-
class
faust.
EventT
(app: faust.types.events._AppT, key: Union[bytes, faust.types.core._ModelT, Any, None], value: Union[bytes, faust.types.core._ModelT, Any], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], message: faust.types.tuples.Message) → None[source]¶ -
app
¶
-
key
¶
-
value
¶
-
headers
¶
-
message
¶
-
acked
¶
-
coroutine
forward
(self, channel: Union[str, faust.types.events._ChannelT], key: Any = None, value: Any = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, channel: Union[str, faust.types.events._ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.
ModelOptions
(*args, **kwargs)[source]¶ -
serializer
= None¶
-
include_metadata
= True¶
-
allow_blessed_key
= False¶
-
isodates
= False¶
-
decimals
= False¶
-
coercions
= None¶
-
fields
= None¶ Flattened view of __annotations__ in MRO order.
- Type
Index
-
fieldset
= None¶ Set of required field names, for fast argument checking.
- Type
Index
-
fieldpos
= None¶ Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.
- Type
Index
-
optionalset
= None¶ Set of optional field names, for fast argument checking.
- Type
Index
-
models
= None¶ Mapping of fields that are ModelT
- Type
Index
-
modelattrs
= None¶
-
field_coerce
= None¶ Mapping of fields that need to be coerced. Key is the name of the field, value is the coercion handler function.
- Type
Index
-
defaults
= None¶ Mapping of field names to default value.
-
initfield
= None¶ Mapping of init field conversion callbacks.
-
-
class
faust.
Record
→ None[source]¶ Describes a model type that is a record (Mapping).
Examples
>>> class LogEvent(Record, serializer='json'): ... severity: str ... message: str ... timestamp: float ... optional_field: str = 'default value'
>>> event = LogEvent( ... severity='error', ... message='Broken pact', ... timestamp=666.0, ... )
>>> event.severity 'error'
>>> serialized = event.dumps() '{"severity": "error", "message": "Broken pact", "timestamp": 666.0}'
>>> restored = LogEvent.loads(serialized) <LogEvent: severity='error', message='Broken pact', timestamp=666.0>
>>> # You can also subclass a Record to create a new record >>> # with additional fields >>> class RemoteLogEvent(LogEvent): ... url: str
>>> # You can also refer to record fields and pass them around: >>> LogEvent.severity >>> <FieldDescriptor: LogEvent.severity (str)>
-
classmethod
from_data
(data: Mapping, *, preferred_type: Type[faust.types.models.ModelT] = None) → faust.models.record.Record[source]¶ - Return type
-
classmethod
-
class
faust.
Monitor
(*, max_avg_history: int = None, max_commit_latency_history: int = None, max_send_latency_history: int = None, max_assignment_latency_history: int = None, messages_sent: int = 0, tables: MutableMapping[str, faust.sensors.monitor.TableState] = None, messages_active: int = 0, events_active: int = 0, messages_received_total: int = 0, messages_received_by_topic: Counter[str] = None, events_total: int = 0, events_by_stream: Counter[faust.types.streams.StreamT] = None, events_by_task: Counter[_asyncio.Task] = None, events_runtime: Deque[float] = None, commit_latency: Deque[float] = None, send_latency: Deque[float] = None, assignment_latency: Deque[float] = None, events_s: int = 0, messages_s: int = 0, events_runtime_avg: float = 0.0, topic_buffer_full: Counter[faust.types.topics.TopicT] = None, rebalances: int = None, rebalance_return_latency: Deque[float] = None, rebalance_end_latency: Deque[float] = None, rebalance_return_avg: float = 0.0, rebalance_end_avg: float = 0.0, time: Callable[float] = <built-in function monotonic>, **kwargs) → None[source]¶ Default Faust Sensor.
This is the default sensor, recording statistics about events, etc.
-
send_errors
= 0¶ Number of produce operations that ended in error.
-
assignments_completed
= 0¶ Number of partition assignments completed.
-
assignments_failed
= 0¶ Number of partitions assignments that failed.
-
max_avg_history
= 100¶ Max number of total run time values to keep to build average.
-
max_commit_latency_history
= 30¶ Max number of commit latency numbers to keep.
-
max_send_latency_history
= 30¶ Max number of send latency numbers to keep.
-
max_assignment_latency_history
= 30¶ Max number of assignment latency numbers to keep.
-
rebalances
= 0¶ Number of rebalances seen by this worker.
-
tables
= None¶ Mapping of tables
-
commit_latency
= None¶ Deque of commit latency values
-
send_latency
= None¶ Deque of send latency values
-
assignment_latency
= None¶ Deque of assignment latency values.
-
rebalance_return_latency
= None¶
-
rebalance_end_latency
= None¶
-
messages_active
= 0¶ Number of messages currently being processed.
-
messages_received_total
= 0¶ Number of messages processed in total.
-
messages_received_by_topic
= None¶ Count of messages received by topic
-
messages_sent
= 0¶ Number of messages sent in total.
-
messages_sent_by_topic
= None¶ Number of messages sent by topic.
-
messages_s
= 0¶ Number of messages being processed this second.
-
events_active
= 0¶ Number of events currently being processed.
-
events_total
= 0¶ Number of events processed in total.
-
events_by_task
= None¶ Count of events processed by task
-
events_by_stream
= None¶ Count of events processed by stream
-
events_s
= 0¶ Number of events being processed this second.
-
events_runtime_avg
= 0.0¶ Average event runtime over the last second.
-
events_runtime
= None¶ Deque of run times used for averages
-
topic_buffer_full
= None¶ Counter of times a topics buffer was full
-
metric_counts
= None¶ Arbitrary counts added by apps
-
tp_committed_offsets
= None¶ Last committed offsets by TopicPartition
-
tp_read_offsets
= None¶ Last read offsets by TopicPartition
-
tp_end_offsets
= None¶ Log end offsets by TopicPartition
-
secs_since
(start_time: float) → float[source]¶ Given timestamp start, return number of seconds since that time.
- Return type
-
ms_since
(start_time: float) → float[source]¶ Given timestamp start, return number of ms since that time.
- Return type
-
logger
= <Logger faust.sensors.monitor (WARNING)>¶
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_tp_commit
(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
-
class
faust.
Sensor
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Base class for sensors.
This sensor does not do anything at all, but can be subclassed to create new monitors.
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Message sent to a stream as an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Event was acknowledged by stream.
Notes
Acknowledged means a stream finished processing the event, but given that multiple streams may be handling the same event, the message cannot be committed before all streams have processed it. When all streams have acknowledged the event, it will go through
on_message_out()
just before offsets are committed.- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
on_rebalance_return
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Consumer replied assignment is done to broker.
- Return type
None
-
on_rebalance_end
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Cluster rebalance fully completed (including recovery).
- Return type
None
-
logger
= <Logger faust.sensors.base (WARNING)>¶
-
-
class
faust.
Codec
(children: Tuple[faust.types.codecs.CodecT, ...] = None, **kwargs) → None[source]¶ Base class for codecs.
-
children
= None¶ next steps in the recursive codec chain.
x = pickle | binary
returns codec with children set to(pickle, binary)
.
-
nodes
= None¶ cached version of children including this codec as the first node. could use chain below, but seems premature so just copying the list.
-
-
class
faust.
Stream
(channel: AsyncIterator[T_co], *, app: faust.types.app.AppT, processors: Iterable[Callable[T]] = None, combined: List[faust.types.streams.JoinableT] = None, on_start: Callable = None, join_strategy: faust.types.joins.JoinT = None, beacon: mode.utils.types.trees.NodeT = None, concurrency_index: int = None, prev: faust.types.streams.StreamT = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, prefix: str = '', loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ A stream: async iterator processing events in channels/topics.
-
logger
= <Logger faust.streams (WARNING)>¶
-
mundane_level
= 'debug'¶
-
get_active_stream
() → faust.types.streams.StreamT[source]¶ Return the currently active stream.
A stream can be derived using
Stream.group_by
etc, so if this stream was used to create another derived stream, this function will return the stream being actively consumed from. E.g. in the example:>>> @app.agent() ... async def agent(a): .. a = a ... b = a.group_by(Withdrawal.account_id) ... c = b.through('backup_topic') ... async for value in c: ... ...
The return value of
a.get_active_stream()
would bec
.Notes
The chain of streams that leads to the active stream is decided by the
_next
attribute. To get to the active stream we just traverse this linked-list:>>> def get_active_stream(self): ... node = self ... while node._next: ... node = node._next
- Return type
StreamT
[+T_co]
-
add_processor
(processor: Callable[T]) → None[source]¶ Add processor callback executed whenever a new event is received.
Processor functions can be async or non-async, must accept a single argument, and should return the value, mutated or not.
For example a processor handling a stream of numbers may modify the value:
def double(value: int) -> int: return value * 2 stream.add_processor(double)
- Return type
None
-
clone
(**kwargs) → faust.types.streams.StreamT[source]¶ Create a clone of this stream.
Notes
If the cloned stream is supposed to supersede this stream, like in
group_by
/through
/etc., you should use_chain()
instead so stream._next = cloned_stream is set andget_active_stream()
returns the cloned stream.- Return type
StreamT
[+T_co]
-
events
() → AsyncIterable[faust.types.events.EventT][source]¶ Iterate over the stream as events exclusively.
This means the stream must be iterating over a channel, or at least an iterable of event objects.
- Return type
-
enumerate
(start: int = 0) → AsyncIterable[Tuple[int, T_co]][source]¶ Enumerate values received on this stream.
Unlike Python’s built-in
enumerate
, this works with async generators.- Return type
AsyncIterable
[Tuple
[int
, +T_co]]
-
through
(channel: Union[str, faust.types.channels.ChannelT]) → faust.types.streams.StreamT[source]¶ Forward values to in this stream to channel.
Send messages received on this stream to another channel, and return a new stream that consumes from that channel.
Notes
The messages are forwarded after any processors have been applied.
Example
topic = app.topic('foo') @app.agent(topic) async def mytask(stream): async for value in stream.through(app.topic('bar')): # value was first received in topic 'foo', # then forwarded and consumed from topic 'bar' print(value)
- Return type
StreamT
[+T_co]
-
echo
(*channels) → faust.types.streams.StreamT[source]¶ Forward values to one or more channels.
Unlike
through()
, we don’t consume from these channels.- Return type
StreamT
[+T_co]
-
group_by
(key: Union[faust.types.models.FieldDescriptorT, Callable[T, Union[bytes, faust.types.core._ModelT, Any, None]]], *, name: str = None, topic: faust.types.topics.TopicT = None, partitions: int = None) → faust.types.streams.StreamT[source]¶ Create new stream that repartitions the stream using a new key.
- Parameters
key (
Union
[FieldDescriptorT
,Callable
[[~T],Union
[bytes
,_ModelT
,Any
,None
]]]) –The key argument decides how the new key is generated, it can be a field descriptor, a callable, or an async callable.
- Note: The
name
argument must be provided if the key argument is a callable.
- Note: The
name (
Optional
[str
]) – Suffix to use for repartitioned topics. This argument is required if key is a callable.
Examples
Using a field descriptor to use a field in the event as the new key:
s = withdrawals_topic.stream() # values in this stream are of type Withdrawal async for event in s.group_by(Withdrawal.account_id): ...
Using an async callable to extract a new key:
s = withdrawals_topic.stream() async def get_key(withdrawal): return await aiohttp.get( f'http://e.com/resolve_account/{withdrawal.account_id}') async for event in s.group_by(get_key): ...
Using a regular callable to extract a new key:
s = withdrawals_topic.stream() def get_key(withdrawal): return withdrawal.account_id.upper() async for event in s.group_by(get_key): ...
- Return type
StreamT
[+T_co]
-
derive_topic
(name: str, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]¶ Create Topic description derived from the K/V type of this stream.
- Parameters
name (
str
) – Topic name.key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Specific key type to use for this topic. If not set, the key type of this stream will be used.value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Specific value type to use for this topic. If not set, the value type of this stream will be used.
- Raises
ValueError – if the stream channel is not a topic.
- Return type
TopicT
[]
-
coroutine
ack
(self, event: faust.types.events.EventT) → bool[source]¶ Ack event.
This will decrease the reference count of the event message by one, and when the reference count reaches zero, the worker will commit the offset so that the message will not be seen by a worker again.
-
items
() → AsyncIterator[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], T_co]][source]¶ Iterate over the stream as
key, value
pairs.Examples
@app.agent(topic) async def mytask(stream): async for key, value in stream.items(): print(key, value)
- Return type
AsyncIterator
[Tuple
[Union
[bytes
,_ModelT
,Any
,None
], +T_co]]
-
coroutine
remove_from_stream
(self, stream: faust.types.streams.StreamT) → None[source]¶ - Return type
None
-
coroutine
send
(self, value: T_contra) → None[source]¶ Send value into stream locally (bypasses topic).
- Return type
None
-
take
(max_: int, within: Union[datetime.timedelta, float, str]) → AsyncIterable[Sequence[T_co]][source]¶ Buffer n values at a time and yield a list of buffered values.
- Parameters
within (
Union
[timedelta
,float
,str
]) – Timeout for when we give up waiting for another value, and process the values we have. Warning: If there’s no timeout (i.e. timeout=None), the agent is likely to stall and block buffered events for an unreasonable length of time(!).- Return type
AsyncIterable
[Sequence
[+T_co]]
-
-
class
faust.
StreamT
(channel: AsyncIterator[T_co] = None, *, app: faust.types.streams._AppT = None, processors: Iterable[Callable[T]] = None, combined: List[faust.types.streams.JoinableT] = None, on_start: Callable = None, join_strategy: faust.types.streams._JoinT = None, beacon: mode.utils.types.trees.NodeT = None, concurrency_index: int = None, prev: Optional[faust.types.streams.StreamT] = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, prefix: str = '', loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
outbox
= None¶
-
join_strategy
= None¶
-
task_owner
= None¶
-
current_event
= None¶
-
active_partitions
= None¶
-
concurrency_index
= None¶
-
enable_acks
= True¶
-
prefix
= ''¶
-
enumerate
(start: int = 0) → AsyncIterable[Tuple[int, T_co]][source]¶ - Return type
AsyncIterable
[Tuple
[int
, +T_co]]
-
through
(channel: Union[str, faust.types.channels.ChannelT]) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
group_by
(key: Union[faust.types.models.FieldDescriptorT, Callable[T, Union[bytes, faust.types.core._ModelT, Any, None]]], *, name: str = None, topic: faust.types.topics.TopicT = None) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
derive_topic
(name: str, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
coroutine
items
(self) → AsyncIterator[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], T_co]][source]¶
-
-
faust.
current_event
() → Optional[faust.types.events.EventT][source]¶ Return the event currently being processed, or None.
-
class
faust.
SetTable
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Table that maintains a dictionary of sets.
-
WindowWrapper
¶ alias of
SetWindowWrapper
-
logger
= <Logger faust.tables.sets (WARNING)>¶
-
-
class
faust.
Table
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Table (non-windowed).
-
class
WindowWrapper
(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None] = None, key_index: bool = False, key_index_table: faust.types.tables.TableT = None) → None¶ Windowed table wrapper.
A windowed table does not return concrete values when keys are accessed, instead
WindowSet
is returned so that the values can be further reduced to the wanted time period.-
ValueType
¶ alias of
WindowSet
-
clone
(relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
get_relative_timestamp
¶
-
key_index
= False¶
-
key_index_table
= None¶
-
on_del_key
(key: Any) → None¶ - Return type
None
-
on_recover
(fun: Callable[Awaitable[None]]) → Callable[Awaitable[None]]¶
-
on_set_key
(key: Any, value: Any) → None¶ - Return type
None
-
relative_to
(ts: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_field
(field: faust.types.models.FieldDescriptorT) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_now
() → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_stream
() → faust.types.tables.WindowWrapperT¶ - Return type
-
values
(event: faust.types.events.EventT = None) → ValuesView¶ - Return type
ValuesView
[+VT_co]
-
-
using_window
(window: faust.types.windows.WindowT, *, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
hopping
(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
tumbling
(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
on_key_set
(key: KT, value: VT) → None[source]¶ Handle that value for a key is being set.
- Return type
None
-
logger
= <Logger faust.tables.table (WARNING)>¶
-
class
-
class
faust.
Topic
(app: faust.types.app.AppT, *, topics: Sequence[str] = None, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, is_iterator: bool = False, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, queue: mode.utils.queues.ThrowableQueue = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Define new topic description.
- Parameters
app (
AppT
[]) – App instance used to create this topic description.partitions (
Optional
[int
]) – Number of partitions for these topics. On declaration, topics are created using this. Note: If a message is produced before the topic is declared, andautoCreateTopics
is enabled on the Kafka Server, the number of partitions used will be specified by the server configuration.retention (
Union
[timedelta
,float
,str
,None
]) – Number of seconds (as float/timedelta
) to keep messages in the topic before they can be expired by the server.pattern (
Union
[str
,Pattern
[AnyStr
],None
]) – Regular expression evaluated to decide what topics to subscribe to. You cannot specify both topics and a pattern.key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – How to deserialize keys for messages in this topic. Can be afaust.Model
type,str
,bytes
, orNone
for “autodetect”value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – How to deserialize values for messages in this topic. Can be afaust.Model
type,str
,bytes
, orNone
for “autodetect”active_partitions (
Optional
[Set
[TP
]]) – Set offaust.types.tuples.TP
that this topic should be restricted to.
- Raises
TypeError – if both topics and pattern is provided.
-
derive
(**kwargs) → faust.types.channels.ChannelT[source]¶ Create new
Topic
derived from this topic.Configuration will be copied from this topic, but any parameter overridden as a keyword argument.
See also
derive_topic()
: for a list of supported keyword arguments.- Return type
ChannelT
[]
-
derive_topic
(*, topics: Sequence[str] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, internal: bool = None, config: Mapping[str, Any] = None, prefix: str = '', suffix: str = '', **kwargs) → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]¶ - Return type
EventT
[]
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to topic.
- Return type
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
class
faust.
TopicT
(app: faust.types.topics._AppT, *, topics: Sequence[str] = None, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: faust.types.topics._ModelArg = None, value_type: faust.types.topics._ModelArg = None, is_iterator: bool = False, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, queue: mode.utils.queues.ThrowableQueue = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
topics
= None¶ Iterable/Sequence of topic names to subscribe to.
-
retention
= None¶ expiry time in seconds for messages in the topic.
- Type
Topic retention setting
-
compacting
= None¶ Flag that when enabled means the topic can be “compacted”: if the topic is a log of key/value pairs, the broker can delete old values for the same key.
-
replicas
= None¶ Number of replicas for topic.
-
config
= None¶ Additional configuration as a mapping.
-
acks
= None¶ Enable acks for this topic.
-
internal
= None¶ it’s owned by us and we are allowed to create or delete the topic as necessary.
- Type
Mark topic as internal
-
pattern
¶ or instead of
topics
, a regular expression used to match topics we want to subscribe to. :rtype:Optional
[Pattern
[AnyStr
]]
-
derive_topic
(*, topics: Sequence[str] = None, key_type: faust.types.topics._ModelArg = None, value_type: faust.types.topics._ModelArg = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, internal: bool = False, config: Mapping[str, Any] = None, prefix: str = '', suffix: str = '', **kwargs) → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
-
class
faust.
GSSAPICredentials
(*, kerberos_service_name: str = 'kafka', kerberos_domain_name: str = None, ssl_context: ssl.SSLContext = None, mechanism: Union[str, faust.types.auth.SASLMechanism] = None) → None[source]¶ Describe GSSAPI credentials over SASL.
-
protocol
= 'SASL_PLAINTEXT'¶
-
mechanism
= 'GSSAPI'¶
-
-
class
faust.
SASLCredentials
(*, username: str = None, password: str = None, ssl_context: ssl.SSLContext = None, mechanism: Union[str, faust.types.auth.SASLMechanism] = None) → None[source]¶ Describe SASL credentials.
-
protocol
= 'SASL_PLAINTEXT'¶
-
mechanism
= 'PLAIN'¶
-
-
class
faust.
SSLCredentials
(context: ssl.SSLContext = None, *, purpose: Any = None, cafile: Optional[str] = None, capath: Optional[str] = None, cadata: Optional[str] = None) → None[source]¶ Describe SSL credentials/settings.
-
protocol
= 'SSL'¶
-
-
class
faust.
Settings
(id: str, *, version: int = None, broker: Union[str, yarl.URL, List[yarl.URL]] = None, broker_client_id: str = None, broker_request_timeout: Union[datetime.timedelta, float, str] = None, broker_credentials: Union[faust.types.auth.CredentialsT, ssl.SSLContext] = None, broker_commit_every: int = None, broker_commit_interval: Union[datetime.timedelta, float, str] = None, broker_commit_livelock_soft_timeout: Union[datetime.timedelta, float, str] = None, broker_session_timeout: Union[datetime.timedelta, float, str] = None, broker_heartbeat_interval: Union[datetime.timedelta, float, str] = None, broker_check_crcs: bool = None, broker_max_poll_records: int = None, agent_supervisor: Union[_T, str] = None, store: Union[str, yarl.URL] = None, cache: Union[str, yarl.URL] = None, web: Union[str, yarl.URL] = None, web_enabled: bool = True, processing_guarantee: Union[str, faust.types.enums.ProcessingGuarantee] = None, timezone: datetime.tzinfo = None, autodiscover: Union[bool, Iterable[str], Callable[Iterable[str]]] = None, origin: str = None, canonical_url: Union[str, yarl.URL] = None, datadir: Union[pathlib.Path, str] = None, tabledir: Union[pathlib.Path, str] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, logging_config: Dict = None, loghandlers: List[logging.Handler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = None, topic_allow_declare: bool = None, id_format: str = None, reply_to: str = None, reply_to_prefix: str = None, reply_create_topic: bool = None, reply_expires: Union[datetime.timedelta, float, str] = None, ssl_context: ssl.SSLContext = None, stream_buffer_maxsize: int = None, stream_wait_empty: bool = None, stream_ack_cancelled_tasks: bool = None, stream_ack_exceptions: bool = None, stream_publish_on_commit: bool = None, stream_recovery_delay: Union[datetime.timedelta, float, str] = None, producer_linger_ms: int = None, producer_max_batch_size: int = None, producer_acks: int = None, producer_max_request_size: int = None, producer_compression_type: str = None, producer_partitioner: Union[_T, str] = None, producer_request_timeout: Union[datetime.timedelta, float, str] = None, producer_api_version: str = None, consumer_max_fetch_size: int = None, consumer_auto_offset_reset: str = None, web_bind: str = None, web_port: int = None, web_host: str = None, web_transport: Union[str, yarl.URL] = None, web_in_thread: bool = None, web_cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[_T, str] = None, ConsumerScheduler: Union[_T, str] = None, Stream: Union[_T, str] = None, Table: Union[_T, str] = None, SetTable: Union[_T, str] = None, TableManager: Union[_T, str] = None, Serializers: Union[_T, str] = None, Worker: Union[_T, str] = None, PartitionAssignor: Union[_T, str] = None, LeaderAssignor: Union[_T, str] = None, Router: Union[_T, str] = None, Topic: Union[_T, str] = None, HttpClient: Union[_T, str] = None, Monitor: Union[_T, str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None[source]¶ -
-
id_format
= '{id}-v{self.version}'¶
-
ssl_context
= None¶
-
autodiscover
= False¶
-
broker_client_id
= 'faust-1.5.5'¶
-
timezone
= datetime.timezone.utc¶
-
broker_commit_every
= 10000¶
-
broker_check_crcs
= True¶
-
key_serializer
= 'raw'¶
-
value_serializer
= 'json'¶
-
table_standby_replicas
= 1¶
-
topic_replication_factor
= 1¶
-
topic_partitions
= 8¶
-
topic_allow_declare
= True¶
-
reply_create_topic
= False¶
-
logging_config
= None¶
-
stream_buffer_maxsize
= 4096¶
-
stream_wait_empty
= True¶
-
stream_ack_cancelled_tasks
= True¶
-
stream_ack_exceptions
= True¶
-
stream_publish_on_commit
= False¶
-
producer_linger_ms
= 0¶
-
producer_max_batch_size
= 16384¶
-
producer_acks
= -1¶
-
producer_max_request_size
= 1000000¶
-
producer_compression_type
= None¶
-
producer_api_version
= 'auto'¶
-
consumer_max_fetch_size
= 4194304¶
-
consumer_auto_offset_reset
= 'earliest'¶
-
web_bind
= '0.0.0.0'¶
-
web_port
= 6066¶
-
web_host
= 'build-8929922-project-230058-faust'¶
-
web_in_thread
= False¶
-
web_cors_options
= None¶
-
worker_redirect_stdouts
= True¶
-
worker_redirect_stdouts_level
= 'WARN'¶
-
reply_to_prefix
= 'f-reply-'¶
-
processing_guarantee
¶ - Return type
-
broker_credentials
¶ - Return type
-
producer_partitioner
¶
-
agent_supervisor
¶ - Return type
-
TableManager
¶ - Return type
-
PartitionAssignor
¶ - Return type
-
LeaderAssignor
¶ - Return type
-
-
faust.
HoppingWindow
¶ alias of
faust.windows._PyHoppingWindow
-
class
faust.
TumblingWindow
(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → None[source]¶ Tumbling window type.
Fixed-size, non-overlapping, gap-less windows.
-
faust.
SlidingWindow
¶ alias of
faust.windows._PySlidingWindow
-
class
faust.
Worker
(app: faust.types.app.AppT, *services, sensors: Iterable[faust.types.sensors.SensorT] = None, debug: bool = False, quiet: bool = False, loglevel: Union[str, int] = None, logfile: Union[str, IO] = None, stdout: IO = <_io.TextIOWrapper name='<stdout>' mode='w' encoding='UTF-8'>, stderr: IO = <_io.TextIOWrapper name='<stderr>' mode='w' encoding='UTF-8'>, blocking_timeout: float = 10.0, workdir: Union[pathlib.Path, str] = None, console_port: int = 50101, loop: asyncio.events.AbstractEventLoop = None, redirect_stdouts: bool = None, redirect_stdouts_level: int = None, logging_config: Dict = None, **kwargs) → None[source]¶ Worker.
- Usage:
You can start a worker using:
the faust worker program.
instantiating Worker programmatically and calling execute_from_commandline():
>>> worker = Worker(app) >>> worker.execute_from_commandline()
or if you already have an event loop, calling
await start
, but in that case you are responsible for gracefully shutting down the event loop:async def start_worker(worker: Worker) -> None: await worker.start() def manage_loop(): loop = asyncio.get_event_loop() worker = Worker(app, loop=loop) try: loop.run_until_complete(start_worker(worker) finally: worker.stop_and_shutdown_loop()
- Parameters
app (
AppT
[]) – The Faust app to start.*services – Services to start with worker. This includes application instances to start.
sensors (Iterable[SensorT]) – List of sensors to include.
debug (bool) – Enables debugging mode [disabled by default].
quiet (bool) – Do not output anything to console [disabled by default].
loglevel (Union[str, int]) – Level to use for logging, can be string (one of: CRIT|ERROR|WARN|INFO|DEBUG), or integer.
logfile (Union[str, IO]) – Name of file or a stream to log to.
stdout (IO) – Standard out stream.
stderr (IO) – Standard err stream.
blocking_timeout (float) – When
debug
is enabled this sets the timeout for detecting that the event loop is blocked.workdir (Union[str, Path]) – Custom working directory for the process that the worker will change into when started. This working directory change is permanent for the process, or until something else changes the working directory again.
loop (asyncio.AbstractEventLoop) – Custom event loop object.
-
logger
= <Logger faust.worker (WARNING)>¶
-
app
= None¶ The Faust app started by this worker.
-
sensors
= None¶ Additional sensors to add to the Faust app.
-
workdir
= None¶ Current working directory. Note that if passed as an argument to Worker, the worker will change to this directory when started.
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
class
faust.
Service
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ An asyncio service that can be started/stopped/restarted.
- Keyword Arguments
beacon (NodeT) – Beacon used to track services in a graph.
loop (asyncio.AbstractEventLoop) – Event loop object.
-
abstract
= False¶
-
class
Diag
(service: mode.types.services.ServiceT) → None¶ Service diagnostics.
This can be used to track what your service is doing. For example if your service is a Kafka consumer with a background thread that commits the offset every 30 seconds, you may want to see when this happens:
DIAG_COMMITTING = 'committing' class Consumer(Service): @Service.task async def _background_commit(self) -> None: while not self.should_stop: await self.sleep(30.0) self.diag.set_flag(DIAG_COMITTING) try: await self._consumer.commit() finally: self.diag.unset_flag(DIAG_COMMITTING)
The above code is setting the flag manually, but you can also use a decorator to accomplish the same thing:
@Service.timer(30.0) async def _background_commit(self) -> None: await self.commit() @Service.transitions_with(DIAG_COMITTING) async def commit(self) -> None: await self._consumer.commit()
-
set_flag
(flag: str) → None¶ - Return type
None
-
unset_flag
(flag: str) → None¶ - Return type
None
-
-
wait_for_shutdown
= False¶ Set to True if .stop must wait for the shutdown flag to be set.
-
shutdown_timeout
= 60.0¶ Time to wait for shutdown flag set before we give up.
-
restart_count
= 0¶ Current number of times this service instance has been restarted.
-
mundane_level
= 'info'¶ The log level for mundane info such as starting, stopping, etc. Set this to
"debug"
for less information.
-
classmethod
from_awaitable
(coro: Awaitable, *, name: str = None, **kwargs) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
classmethod
task
(fun: Callable[Any, Awaitable[None]]) → mode.services.ServiceTask[source]¶ Decorate function to be used as background task.
Example
>>> class S(Service): ... ... @Service.task ... async def background_task(self): ... while not self.should_stop: ... await self.sleep(1.0) ... print('Waking up')
- Return type
ServiceTask
-
classmethod
timer
(interval: Union[datetime.timedelta, float, str]) → Callable[Callable[mode.types.services.ServiceT, Awaitable[None]], mode.services.ServiceTask][source]¶ Background timer executing every
n
seconds.Example
>>> class S(Service): ... ... @Service.timer(1.0) ... async def background_timer(self): ... print('Waking up')
-
classmethod
transitions_to
(flag: str) → Callable[source]¶ Decorate function to set and reset diagnostic flag.
- Return type
-
add_dependency
(service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]¶ Add dependency to other service.
The service will be started/stopped with this service.
- Return type
ServiceT
[]
-
add_future
(coro: Awaitable) → _asyncio.Future[source]¶ Add relationship to asyncio.Future.
The future will be joined when this service is stopped.
- Return type
Future
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
coroutine
add_runtime_dependency
(self, service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
coroutine
crash
(self, reason: BaseException) → None[source]¶ Crash the service and all child services.
- Return type
None
-
itertimer
(interval: Union[datetime.timedelta, float, str], *, max_drift_correction: float = 0.1, loop: asyncio.events.AbstractEventLoop = None, sleep: Callable[..., Awaitable] = None, clock: Callable[float] = <built-in function perf_counter>, name: str = '') → AsyncIterator[float][source]¶ Sleep
interval
seconds for every iteration.This is an async iterator that takes advantage of
timer_intervals()
to act as a timer that stop drift from occurring, and adds a tiny amount of drift to timers so that they don’t start at the same time.Uses
Service.sleep
which will bail-out-quick if the service is stopped.Note
Will sleep the full interval seconds before returning from first iteration.
Examples
>>> async for sleep_time in self.itertimer(1.0): ... print('another second passed, just woke up...') ... await perform_some_http_request()
- Return type
-
coroutine
join_services
(self, services: Sequence[mode.types.services.ServiceT]) → None[source]¶ - Return type
None
-
logger
= <Logger mode.services (WARNING)>¶
-
coroutine
maybe_start
(self) → None[source]¶ Start the service, if it has not already been started.
- Return type
None
-
coroutine
sleep
(self, n: Union[datetime.timedelta, float, str], *, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Sleep for
n
seconds, or until service stopped.- Return type
None
-
coroutine
transition_with
(self, flag: str, fut: Awaitable, *args, **kwargs) → Any[source]¶ - Return type
-
coroutine
wait
(self, *coros, timeout: Union[datetime.timedelta, float, str] = None) → mode.services.WaitResult[source]¶ Wait for coroutines to complete, or until the service stops.
- Return type
WaitResult
-
coroutine
wait_first
(self, *coros, timeout: Union[datetime.timedelta, float, str] = None) → mode.services.WaitResults[source]¶ - Return type
WaitResults
-
coroutine
wait_for_stopped
(self, *coros, timeout: Union[datetime.timedelta, float, str] = None) → bool[source]¶ - Return type
-
coroutine
wait_many
(self, coros: Iterable[Union[Generator[[Any, None], Any], Awaitable, asyncio.locks.Event, mode.utils.locks.Event]], *, timeout: Union[datetime.timedelta, float, str] = None) → mode.services.WaitResult[source]¶ - Return type
WaitResult
-
coroutine
wait_until_stopped
(self) → None[source]¶ Wait until the service is signalled to stop.
- Return type
None
-
set_shutdown
() → None[source]¶ Set the shutdown signal.
Notes
If
wait_for_shutdown
is set, stopping the service will wait for this flag to be set.- Return type
None
-
class
faust.
ServiceT
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Abstract type for an asynchronous service that can be started/stopped.
See also
-
wait_for_shutdown
= False¶
-
restart_count
= 0¶
-
supervisor
= None¶
-
add_dependency
(service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
coroutine
add_runtime_dependency
(self, service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
loop
¶ - Return type
AbstractEventLoop
-
faust.auth
¶
Authentication Credentials.
-
class
faust.auth.
SASLCredentials
(*, username: str = None, password: str = None, ssl_context: ssl.SSLContext = None, mechanism: Union[str, faust.types.auth.SASLMechanism] = None) → None[source]¶ Describe SASL credentials.
-
protocol
= 'SASL_PLAINTEXT'¶
-
mechanism
= 'PLAIN'¶
-
-
class
faust.auth.
GSSAPICredentials
(*, kerberos_service_name: str = 'kafka', kerberos_domain_name: str = None, ssl_context: ssl.SSLContext = None, mechanism: Union[str, faust.types.auth.SASLMechanism] = None) → None[source]¶ Describe GSSAPI credentials over SASL.
-
protocol
= 'SASL_PLAINTEXT'¶
-
mechanism
= 'GSSAPI'¶
-
faust.exceptions
¶
Faust exceptions.
-
exception
faust.exceptions.
AlreadyConfiguredWarning
[source]¶ Trying to configure app after configuration accessed.
-
exception
faust.exceptions.
ImproperlyConfigured
[source]¶ The library is not configured/installed correctly.
-
exception
faust.exceptions.
DecodeError
[source]¶ Error while decoding/deserializing message key/value.
-
exception
faust.exceptions.
ValueDecodeError
[source]¶ Error while decoding/deserializing message value.
-
exception
faust.exceptions.
SameNode
[source]¶ Exception raised by router when data is located on same node.
-
exception
faust.exceptions.
ProducerSendError
[source]¶ Error while sending attached messages prior to commit.
faust.channels
¶
Channel.
A channel is used to send values to streams.
The stream will iterate over incoming events in the channel.
-
class
faust.channels.
Channel
(app: faust.types.app.AppT, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, is_iterator: bool = False, queue: mode.utils.queues.ThrowableQueue = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Create new channel.
- Parameters
app (
AppT
[]) – The app that created this channel (app.channel()
)key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – The Model used for keys in this channel.value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – The Model used for values in this channel.maxsize (
Optional
[int
]) – The maximum number of messages this channel can hold. If exceeded any newput
call will block until a message is removed from the channel.loop (
Optional
[AbstractEventLoop
]) – Theasyncio
event loop to use.
-
queue
¶ - Return type
-
clone
(*, is_iterator: bool = None, **kwargs) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
clone_using_queue
(queue: asyncio.queues.Queue) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
stream
(**kwargs) → faust.types.streams.StreamT[source]¶ Create stream reading from this channel.
- Return type
StreamT
[+T_co]
-
as_future_message
(key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → faust.types.tuples.FutureMessage[source]¶ - Return type
-
prepare_headers
(headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None]) → Union[List[Tuple[str, bytes]], MutableMapping[str, bytes]][source]¶
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
prepare_value
(value: Union[bytes, faust.types.core._ModelT, Any], value_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]¶ - Return type
EventT
[]
-
coroutine
get
(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]¶ - Return type
-
coroutine
on_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_key_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_value_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to channel.
- Return type
faust.events
¶
Events received in streams.
-
class
faust.events.
Event
(app: faust.types.app.AppT, key: Union[bytes, faust.types.core._ModelT, Any, None], value: Union[bytes, faust.types.core._ModelT, Any], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], message: faust.types.tuples.Message) → None[source]¶ An event received on a channel.
Notes
Events have a key and a value:
event.key, event.value
They also have a reference to the original message (if available), such as a Kafka record:
event.message.offset
Iterating over channels/topics yields Event:
- async for event in channel:
…
Iterating over a stream (that in turn iterate over channel) yields Event.value:
async for value in channel.stream() # value is event.value ...
If you only have a Stream object, you can also access underlying events by using
Stream.events
.For example:
async for event in channel.stream.events(): ...
Also commonly used for finding the “current event” related to a value in the stream:
stream = channel.stream() async for event in stream.events(): event = stream.current_event message = event.message topic = event.message.topic
You can retrieve the current event in a stream to:
Get access to the serialized key+value.
Get access to message properties like, what topic+partition the value was received on, or its offset.
If you want access to both key and value, you should use
stream.items()
instead.async for key, value in stream.items(): ...
stream.current_event
can also be accessed but you must take extreme care you are using the correct stream object. Methods such as.group_by(key)
and.through(topic)
returns cloned stream objects, so in the example:The best way to access the current_event in an agent is to use the
ContextVar
:from faust import current_event @app.agent(topic) async def process(stream): async for value in stream: event = current_event()
-
coroutine
forward
(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = <object object>, value: Union[bytes, faust.types.core._ModelT, Any] = <object object>, partition: int = None, timestamp: float = None, headers: Any = <object object>, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Forward original message (will not be reserialized).
- Return type
-
coroutine
send
(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = <object object>, value: Union[bytes, faust.types.core._ModelT, Any] = <object object>, partition: int = None, timestamp: float = None, headers: Any = <object object>, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send object to channel.
- Return type
faust.joins
¶
Join strategies.
-
class
faust.joins.
Join
(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]¶ Base class for join strategies.
-
class
faust.joins.
RightJoin
(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]¶ Right-join strategy.
-
class
faust.joins.
LeftJoin
(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]¶ Left-join strategy.
faust.streams
¶
Streams.
-
faust.streams.
current_event
() → Optional[faust.types.events.EventT][source]¶ Return the event currently being processed, or None.
-
class
faust.streams.
Stream
(channel: AsyncIterator[T_co], *, app: faust.types.app.AppT, processors: Iterable[Callable[T]] = None, combined: List[faust.types.streams.JoinableT] = None, on_start: Callable = None, join_strategy: faust.types.joins.JoinT = None, beacon: mode.utils.types.trees.NodeT = None, concurrency_index: int = None, prev: faust.types.streams.StreamT = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, prefix: str = '', loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ A stream: async iterator processing events in channels/topics.
-
logger
= <Logger faust.streams (WARNING)>¶
-
mundane_level
= 'debug'¶
-
get_active_stream
() → faust.types.streams.StreamT[source]¶ Return the currently active stream.
A stream can be derived using
Stream.group_by
etc, so if this stream was used to create another derived stream, this function will return the stream being actively consumed from. E.g. in the example:>>> @app.agent() ... async def agent(a): .. a = a ... b = a.group_by(Withdrawal.account_id) ... c = b.through('backup_topic') ... async for value in c: ... ...
The return value of
a.get_active_stream()
would bec
.Notes
The chain of streams that leads to the active stream is decided by the
_next
attribute. To get to the active stream we just traverse this linked-list:>>> def get_active_stream(self): ... node = self ... while node._next: ... node = node._next
- Return type
StreamT
[+T_co]
-
add_processor
(processor: Callable[T]) → None[source]¶ Add processor callback executed whenever a new event is received.
Processor functions can be async or non-async, must accept a single argument, and should return the value, mutated or not.
For example a processor handling a stream of numbers may modify the value:
def double(value: int) -> int: return value * 2 stream.add_processor(double)
- Return type
None
-
clone
(**kwargs) → faust.types.streams.StreamT[source]¶ Create a clone of this stream.
Notes
If the cloned stream is supposed to supersede this stream, like in
group_by
/through
/etc., you should use_chain()
instead so stream._next = cloned_stream is set andget_active_stream()
returns the cloned stream.- Return type
StreamT
[+T_co]
-
events
() → AsyncIterable[faust.types.events.EventT][source]¶ Iterate over the stream as events exclusively.
This means the stream must be iterating over a channel, or at least an iterable of event objects.
- Return type
-
enumerate
(start: int = 0) → AsyncIterable[Tuple[int, T_co]][source]¶ Enumerate values received on this stream.
Unlike Python’s built-in
enumerate
, this works with async generators.- Return type
AsyncIterable
[Tuple
[int
, +T_co]]
-
through
(channel: Union[str, faust.types.channels.ChannelT]) → faust.types.streams.StreamT[source]¶ Forward values to in this stream to channel.
Send messages received on this stream to another channel, and return a new stream that consumes from that channel.
Notes
The messages are forwarded after any processors have been applied.
Example
topic = app.topic('foo') @app.agent(topic) async def mytask(stream): async for value in stream.through(app.topic('bar')): # value was first received in topic 'foo', # then forwarded and consumed from topic 'bar' print(value)
- Return type
StreamT
[+T_co]
-
echo
(*channels) → faust.types.streams.StreamT[source]¶ Forward values to one or more channels.
Unlike
through()
, we don’t consume from these channels.- Return type
StreamT
[+T_co]
-
group_by
(key: Union[faust.types.models.FieldDescriptorT, Callable[T, Union[bytes, faust.types.core._ModelT, Any, None]]], *, name: str = None, topic: faust.types.topics.TopicT = None, partitions: int = None) → faust.types.streams.StreamT[source]¶ Create new stream that repartitions the stream using a new key.
- Parameters
key (
Union
[FieldDescriptorT
,Callable
[[~T],Union
[bytes
,_ModelT
,Any
,None
]]]) –The key argument decides how the new key is generated, it can be a field descriptor, a callable, or an async callable.
- Note: The
name
argument must be provided if the key argument is a callable.
- Note: The
name (
Optional
[str
]) – Suffix to use for repartitioned topics. This argument is required if key is a callable.
Examples
Using a field descriptor to use a field in the event as the new key:
s = withdrawals_topic.stream() # values in this stream are of type Withdrawal async for event in s.group_by(Withdrawal.account_id): ...
Using an async callable to extract a new key:
s = withdrawals_topic.stream() async def get_key(withdrawal): return await aiohttp.get( f'http://e.com/resolve_account/{withdrawal.account_id}') async for event in s.group_by(get_key): ...
Using a regular callable to extract a new key:
s = withdrawals_topic.stream() def get_key(withdrawal): return withdrawal.account_id.upper() async for event in s.group_by(get_key): ...
- Return type
StreamT
[+T_co]
-
derive_topic
(name: str, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]¶ Create Topic description derived from the K/V type of this stream.
- Parameters
name (
str
) – Topic name.key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Specific key type to use for this topic. If not set, the key type of this stream will be used.value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Specific value type to use for this topic. If not set, the value type of this stream will be used.
- Raises
ValueError – if the stream channel is not a topic.
- Return type
TopicT
[]
-
coroutine
ack
(self, event: faust.types.events.EventT) → bool[source]¶ Ack event.
This will decrease the reference count of the event message by one, and when the reference count reaches zero, the worker will commit the offset so that the message will not be seen by a worker again.
-
items
() → AsyncIterator[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], T_co]][source]¶ Iterate over the stream as
key, value
pairs.Examples
@app.agent(topic) async def mytask(stream): async for key, value in stream.items(): print(key, value)
- Return type
AsyncIterator
[Tuple
[Union
[bytes
,_ModelT
,Any
,None
], +T_co]]
-
coroutine
remove_from_stream
(self, stream: faust.types.streams.StreamT) → None[source]¶ - Return type
None
-
coroutine
send
(self, value: T_contra) → None[source]¶ Send value into stream locally (bypasses topic).
- Return type
None
-
take
(max_: int, within: Union[datetime.timedelta, float, str]) → AsyncIterable[Sequence[T_co]][source]¶ Buffer n values at a time and yield a list of buffered values.
- Parameters
within (
Union
[timedelta
,float
,str
]) – Timeout for when we give up waiting for another value, and process the values we have. Warning: If there’s no timeout (i.e. timeout=None), the agent is likely to stall and block buffered events for an unreasonable length of time(!).- Return type
AsyncIterable
[Sequence
[+T_co]]
-
faust.topics
¶
Topic - Named channel using Kafka.
-
class
faust.topics.
Topic
(app: faust.types.app.AppT, *, topics: Sequence[str] = None, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, is_iterator: bool = False, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, queue: mode.utils.queues.ThrowableQueue = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Define new topic description.
- Parameters
app (
AppT
[]) – App instance used to create this topic description.partitions (
Optional
[int
]) – Number of partitions for these topics. On declaration, topics are created using this. Note: If a message is produced before the topic is declared, andautoCreateTopics
is enabled on the Kafka Server, the number of partitions used will be specified by the server configuration.retention (
Union
[timedelta
,float
,str
,None
]) – Number of seconds (as float/timedelta
) to keep messages in the topic before they can be expired by the server.pattern (
Union
[str
,Pattern
[AnyStr
],None
]) – Regular expression evaluated to decide what topics to subscribe to. You cannot specify both topics and a pattern.key_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – How to deserialize keys for messages in this topic. Can be afaust.Model
type,str
,bytes
, orNone
for “autodetect”value_type (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – How to deserialize values for messages in this topic. Can be afaust.Model
type,str
,bytes
, orNone
for “autodetect”active_partitions (
Optional
[Set
[TP
]]) – Set offaust.types.tuples.TP
that this topic should be restricted to.
- Raises
TypeError – if both topics and pattern is provided.
-
derive
(**kwargs) → faust.types.channels.ChannelT[source]¶ Create new
Topic
derived from this topic.Configuration will be copied from this topic, but any parameter overridden as a keyword argument.
See also
derive_topic()
: for a list of supported keyword arguments.- Return type
ChannelT
[]
-
derive_topic
(*, topics: Sequence[str] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, internal: bool = None, config: Mapping[str, Any] = None, prefix: str = '', suffix: str = '', **kwargs) → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]¶ - Return type
EventT
[]
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to topic.
- Return type
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
faust.windows
¶
Window Types.
-
faust.windows.
HoppingWindow
¶ alias of
faust.windows._PyHoppingWindow
-
class
faust.windows.
TumblingWindow
(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → None[source]¶ Tumbling window type.
Fixed-size, non-overlapping, gap-less windows.
-
faust.windows.
SlidingWindow
¶ alias of
faust.windows._PySlidingWindow
faust.worker
¶
Worker.
A “worker” starts a single instance of a Faust application.
See also
Starting the App: for more information.
-
class
faust.worker.
Worker
(app: faust.types.app.AppT, *services, sensors: Iterable[faust.types.sensors.SensorT] = None, debug: bool = False, quiet: bool = False, loglevel: Union[str, int] = None, logfile: Union[str, IO] = None, stdout: IO = <_io.TextIOWrapper name='<stdout>' mode='w' encoding='UTF-8'>, stderr: IO = <_io.TextIOWrapper name='<stderr>' mode='w' encoding='UTF-8'>, blocking_timeout: float = 10.0, workdir: Union[pathlib.Path, str] = None, console_port: int = 50101, loop: asyncio.events.AbstractEventLoop = None, redirect_stdouts: bool = None, redirect_stdouts_level: int = None, logging_config: Dict = None, **kwargs) → None[source]¶ Worker.
- Usage:
You can start a worker using:
the faust worker program.
instantiating Worker programmatically and calling execute_from_commandline():
>>> worker = Worker(app) >>> worker.execute_from_commandline()
or if you already have an event loop, calling
await start
, but in that case you are responsible for gracefully shutting down the event loop:async def start_worker(worker: Worker) -> None: await worker.start() def manage_loop(): loop = asyncio.get_event_loop() worker = Worker(app, loop=loop) try: loop.run_until_complete(start_worker(worker) finally: worker.stop_and_shutdown_loop()
- Parameters
app (
AppT
[]) – The Faust app to start.*services – Services to start with worker. This includes application instances to start.
sensors (Iterable[SensorT]) – List of sensors to include.
debug (bool) – Enables debugging mode [disabled by default].
quiet (bool) – Do not output anything to console [disabled by default].
loglevel (Union[str, int]) – Level to use for logging, can be string (one of: CRIT|ERROR|WARN|INFO|DEBUG), or integer.
logfile (Union[str, IO]) – Name of file or a stream to log to.
stdout (IO) – Standard out stream.
stderr (IO) – Standard err stream.
blocking_timeout (float) – When
debug
is enabled this sets the timeout for detecting that the event loop is blocked.workdir (Union[str, Path]) – Custom working directory for the process that the worker will change into when started. This working directory change is permanent for the process, or until something else changes the working directory again.
loop (asyncio.AbstractEventLoop) – Custom event loop object.
-
logger
= <Logger faust.worker (WARNING)>¶
-
app
= None¶ The Faust app started by this worker.
-
sensors
= None¶ Additional sensors to add to the Faust app.
-
workdir
= None¶ Current working directory. Note that if passed as an argument to Worker, the worker will change to this directory when started.
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
App¶
faust.app
¶
-
class
faust.app.
App
(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = None, beacon: mode.utils.types.trees.NodeT = None, **options) → None[source]¶ Faust Application.
- Parameters
id (str) – Application ID.
- Keyword Arguments
loop (asyncio.AbstractEventLoop) – optional event loop to use.
See also
Application Parameters – for supported keyword arguments.
-
class
BootStrategy
(app: faust.types.app.AppT, *, enable_web: bool = None, enable_kafka: bool = None, enable_kafka_producer: bool = None, enable_kafka_consumer: bool = None, enable_sensors: bool = None) → None¶ App startup strategy.
The startup strategy defines the graph of services to start when the Faust worker for an app starts.
-
enable_kafka
= True¶
-
enable_kafka_consumer
= None¶
-
enable_kafka_producer
= None¶
-
enable_sensors
= True¶
-
enable_web
= None¶
-
-
class
Settings
(id: str, *, version: int = None, broker: Union[str, yarl.URL, List[yarl.URL]] = None, broker_client_id: str = None, broker_request_timeout: Union[datetime.timedelta, float, str] = None, broker_credentials: Union[faust.types.auth.CredentialsT, ssl.SSLContext] = None, broker_commit_every: int = None, broker_commit_interval: Union[datetime.timedelta, float, str] = None, broker_commit_livelock_soft_timeout: Union[datetime.timedelta, float, str] = None, broker_session_timeout: Union[datetime.timedelta, float, str] = None, broker_heartbeat_interval: Union[datetime.timedelta, float, str] = None, broker_check_crcs: bool = None, broker_max_poll_records: int = None, agent_supervisor: Union[_T, str] = None, store: Union[str, yarl.URL] = None, cache: Union[str, yarl.URL] = None, web: Union[str, yarl.URL] = None, web_enabled: bool = True, processing_guarantee: Union[str, faust.types.enums.ProcessingGuarantee] = None, timezone: datetime.tzinfo = None, autodiscover: Union[bool, Iterable[str], Callable[Iterable[str]]] = None, origin: str = None, canonical_url: Union[str, yarl.URL] = None, datadir: Union[pathlib.Path, str] = None, tabledir: Union[pathlib.Path, str] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, logging_config: Dict = None, loghandlers: List[logging.Handler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = None, topic_allow_declare: bool = None, id_format: str = None, reply_to: str = None, reply_to_prefix: str = None, reply_create_topic: bool = None, reply_expires: Union[datetime.timedelta, float, str] = None, ssl_context: ssl.SSLContext = None, stream_buffer_maxsize: int = None, stream_wait_empty: bool = None, stream_ack_cancelled_tasks: bool = None, stream_ack_exceptions: bool = None, stream_publish_on_commit: bool = None, stream_recovery_delay: Union[datetime.timedelta, float, str] = None, producer_linger_ms: int = None, producer_max_batch_size: int = None, producer_acks: int = None, producer_max_request_size: int = None, producer_compression_type: str = None, producer_partitioner: Union[_T, str] = None, producer_request_timeout: Union[datetime.timedelta, float, str] = None, producer_api_version: str = None, consumer_max_fetch_size: int = None, consumer_auto_offset_reset: str = None, web_bind: str = None, web_port: int = None, web_host: str = None, web_transport: Union[str, yarl.URL] = None, web_in_thread: bool = None, web_cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[_T, str] = None, ConsumerScheduler: Union[_T, str] = None, Stream: Union[_T, str] = None, Table: Union[_T, str] = None, SetTable: Union[_T, str] = None, TableManager: Union[_T, str] = None, Serializers: Union[_T, str] = None, Worker: Union[_T, str] = None, PartitionAssignor: Union[_T, str] = None, LeaderAssignor: Union[_T, str] = None, Router: Union[_T, str] = None, Topic: Union[_T, str] = None, HttpClient: Union[_T, str] = None, Monitor: Union[_T, str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None¶ -
-
LeaderAssignor
¶ - Return type
-
PartitionAssignor
¶ - Return type
-
TableManager
¶ - Return type
-
agent_supervisor
¶ - Return type
-
autodiscover
= False¶
-
broker_check_crcs
= True¶
-
broker_client_id
= 'faust-1.5.5'¶
-
broker_commit_every
= 10000¶
-
broker_credentials
¶ - Return type
-
consumer_auto_offset_reset
= 'earliest'¶
-
consumer_max_fetch_size
= 4194304¶
-
id_format
= '{id}-v{self.version}'¶
-
key_serializer
= 'raw'¶
-
logging_config
= None¶
-
processing_guarantee
¶ - Return type
-
producer_acks
= -1¶
-
producer_api_version
= 'auto'¶
-
producer_compression_type
= None¶
-
producer_linger_ms
= 0¶
-
producer_max_batch_size
= 16384¶
-
producer_max_request_size
= 1000000¶
-
producer_partitioner
¶
-
reply_create_topic
= False¶
-
reply_to_prefix
= 'f-reply-'¶
-
ssl_context
= None¶
-
stream_ack_cancelled_tasks
= True¶
-
stream_ack_exceptions
= True¶
-
stream_buffer_maxsize
= 4096¶
-
stream_publish_on_commit
= False¶
-
stream_wait_empty
= True¶
-
table_standby_replicas
= 1¶
-
timezone
= datetime.timezone.utc¶
-
topic_allow_declare
= True¶
-
topic_partitions
= 8¶
-
topic_replication_factor
= 1¶
-
value_serializer
= 'json'¶
-
web_bind
= '0.0.0.0'¶
-
web_cors_options
= None¶
-
web_host
= 'build-8929922-project-230058-faust'¶
-
web_in_thread
= False¶
-
web_port
= 6066¶
-
worker_redirect_stdouts
= True¶
-
worker_redirect_stdouts_level
= 'WARN'¶
-
-
client_only
= False¶ Set this to True if app should only start the services required to operate as an RPC client (producer and simple reply consumer).
-
producer_only
= False¶ Set this to True if app should run without consumer/tables.
-
tracer
= None¶ Optional tracing support.
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
config_from_object
(obj: Any, *, silent: bool = False, force: bool = False) → None[source]¶ Read configuration from object.
Object is either an actual object or the name of a module to import.
Examples
>>> app.config_from_object('myproj.faustconfig')
>>> from myproj import faustconfig >>> app.config_from_object(faustconfig)
-
discover
(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[Any] = [<built-in method search of _sre.SRE_Pattern object>, '.__main__']) → None[source]¶ - Return type
None
-
topic
(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, maxsize: int = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → faust.types.topics.TopicT[source]¶ Create topic description.
Topics are named channels (for example a Kafka topic), that exist on a server. To make an ephemeral local communication channel use:
channel()
.See also
- Return type
TopicT
[]
-
channel
(*, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, maxsize: int = None, loop: asyncio.events.AbstractEventLoop = None) → faust.types.channels.ChannelT[source]¶ Create new channel.
By default this will create an in-memory channel used for intra-process communication, but in practice channels can be backed by any transport (network or even means of inter-process communication).
See also
- Return type
ChannelT
[]
-
agent
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT][source]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
actor
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
task
(fun: Union[Callable[AppT, Awaitable], Callable[Awaitable]] = None, *, on_leader: bool = False, traced: bool = True) → Union[Callable[Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]], Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]]], Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]][source]¶ Define an async def function to be started with the app.
This is like
timer()
but a one-shot task only executed at worker startup (after recovery and the worker is fully ready for operation).The function may take zero, or one argument. If the target function takes an argument, the
app
argument is passed:>>> @app.task >>> async def on_startup(app): ... print('STARTING UP: %r' % (app,))
Nullary functions are also supported:
>>> @app.task >>> async def on_startup(): ... print('STARTING UP')
-
timer
(interval: Union[datetime.timedelta, float, str], on_leader: bool = False, traced: bool = True, name: str = None, max_drift_correction: float = 0.1) → Callable[source]¶ Define an async def function to be run at periodic intervals.
Like
task()
, but executes periodically until the worker is shut down.This decorator takes an async function and adds it to a list of timers started with the app.
- Parameters
interval (Seconds) – How often the timer executes in seconds.
on_leader (bool) – Should the timer only run on the leader?
Example
>>> @app.timer(interval=10.0) >>> async def every_10_seconds(): ... print('TEN SECONDS JUST PASSED')
>>> app.timer(interval=5.0, on_leader=True) >>> async def every_5_seconds(): ... print('FIVE SECONDS JUST PASSED. ALSO, I AM THE LEADER!')
- Return type
-
crontab
(cron_format: str, *, timezone: datetime.tzinfo = None, on_leader: bool = False, traced: bool = True) → Callable[source]¶ Define periodic task using Crontab description.
This is an
async def
function to be run at the fixed times, defined by the Cron format.Like
timer()
, but executes at fixed times instead of executing at certain intervals.This decorator takes an async function and adds it to a list of Cronjobs started with the app.
- Parameters
cron_format (
str
) – The Cron spec defining fixed times to run the decorated function.- Keyword Arguments
timezone – The timezone to be taken into account for the Cron jobs. If not set value from
timezone
will be taken.on_leader – Should the Cron job only run on the leader?
Example
>>> @app.crontab(cron_format='30 18 * * *', timezone=pytz.timezone('US/Pacific')) >>> async def every_6_30_pm_pacific(): ... print('IT IS 6:30pm')
>>> app.crontab(cron_format='30 18 * * *', on_leader=True) >>> async def every_6_30_pm(): ... print('6:30pm UTC; ALSO, I AM THE LEADER!')
- Return type
-
service
(cls: Type[mode.types.services.ServiceT]) → Type[mode.types.services.ServiceT][source]¶ Decorate
mode.Service
to be started with the app.Examples
from mode import Service @app.service class Foo(Service): ...
-
stream
(channel: Union[AsyncIterable, Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]¶ Create new stream from channel/topic/iterable/async iterable.
- Parameters
channel (
Union
[AsyncIterable
[+T_co],Iterable
[+T_co]]) – Iterable to stream over (async or non-async).kwargs (
Any
) – SeeStream
.
- Return type
StreamT
[+T_co]- Returns
to iterate over events in the stream.
- Return type
-
Table
(name: str, *, default: Callable[Any] = None, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ Define new table.
- Parameters
name (
str
) – Name used for table, note that two tables living in the same application cannot have the same name.default (
Optional
[Callable
[[],Any
]]) – A callable, or type that will return a default value for keys missing in this table.window (
Optional
[WindowT
]) – A windowing strategy to wrap this window in.
Examples
>>> table = app.Table('user_to_amount', default=int) >>> table['George'] 0 >>> table['Elaine'] += 1 >>> table['Elaine'] += 1 >>> table['Elaine'] 2
- Return type
TableT
[~KT, ~VT]
-
SetTable
(name: str, *, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ - Return type
TableT
[~KT, ~VT]
-
page
(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, name: str = None) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Type[faust.web.views.View]][source]¶
-
table_route
(table: faust.types.tables.CollectionT, shard_param: str = None, *, query_param: str = None, match_info: str = None) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
command
(*options, base: Optional[Type[faust.app.base._AppCommand]] = None, **kwargs) → Callable[Callable, Type[faust.app.base._AppCommand]][source]¶
-
trace
(name: str, trace_enabled: bool = True, **extra_context) → ContextManager[source]¶ - Return type
ContextManager
[+T_co]
-
traced
(fun: Callable, name: str = None, sample_rate: float = 1.0, **context) → Callable[source]¶ - Return type
-
FlowControlQueue
(maxsize: int = None, *, clear_on_resume: bool = False, loop: asyncio.events.AbstractEventLoop = None) → mode.utils.queues.ThrowableQueue[source]¶ Like
asyncio.Queue
, but can be suspended/resumed.- Return type
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ Commit offset for acked messages in specified topics’.
Warning
This will commit acked messages in all topics if the topics argument is passed in as
None
.- Return type
-
logger
= <Logger faust.app.base (WARNING)>¶
-
coroutine
maybe_start_client
(self) → None[source]¶ Start the app in Client-Only mode if not started as Server.
- Return type
None
-
coroutine
on_first_start
(self) → None[source]¶ Service started for the first time in this process.
- Return type
None
-
coroutine
on_init_extra_service
(self, service: Union[mode.types.services.ServiceT, Type[mode.types.services.ServiceT]]) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
coroutine
send
(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send event to channel/topic.
- Parameters
channel (
Union
[ChannelT
[],str
]) – Channel/topic or the name of a topic to send event to.partition (
Optional
[int
]) – Specific partition to send to. If not set the partition will be chosen by the partitioner.timestamp (
Optional
[float
]) – Epoch seconds (from Jan 1 1970 UTC) to use as the message timestamp. Defaults to current time.headers (
Union
[List
[Tuple
[str
,bytes
]],Mapping
[str
,bytes
],None
]) – Mapping of key/value pairs, or iterable of key value pairs to use as headers for the message.key_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).value_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).callback (
Optional
[Callable
[[FutureMessage
[]],Union
[None
,Awaitable
[None
]]]]) –Called after the message is fully delivered to the channel, but not to the consumer. Signature must be unary as the
FutureMessage
future is passed to it.The resulting
faust.types.tuples.RecordMetadata
object is then available asfut.result()
.
- Return type
-
coroutine
start_client
(self) → None[source]¶ Start the app in Client-Only mode necessary for RPC requests.
Notes
Once started as a client the app cannot be restarted as Server.
- Return type
None
-
transport
¶ Message transport. :rtype:
TransportT
-
cache
¶ - Return type
-
topics
[source]¶ Topic Conductor.
This is the mediator that moves messages fetched by the Consumer into the streams.
It’s also a set of registered topics by string topic name, so you can check if a topic is being consumed from by doing
topic in app.topics
.
-
flow_control
[source]¶ Flow control of streams.
This object controls flow into stream queues, and can also clear all buffers.
-
http_client
¶ HTTP Client Session. :rtype:
ClientSession
-
class
faust.app.
BootStrategy
(app: faust.types.app.AppT, *, enable_web: bool = None, enable_kafka: bool = None, enable_kafka_producer: bool = None, enable_kafka_consumer: bool = None, enable_sensors: bool = None) → None[source]¶ App startup strategy.
The startup strategy defines the graph of services to start when the Faust worker for an app starts.
-
enable_kafka
= True¶
-
enable_kafka_producer
= None¶
-
enable_kafka_consumer
= None¶
-
enable_web
= None¶
-
enable_sensors
= True¶
-
faust.app.base
¶
Faust Application.
An app is an instance of the Faust library. Everything starts here.
-
class
faust.app.base.
BootStrategy
(app: faust.types.app.AppT, *, enable_web: bool = None, enable_kafka: bool = None, enable_kafka_producer: bool = None, enable_kafka_consumer: bool = None, enable_sensors: bool = None) → None[source]¶ App startup strategy.
The startup strategy defines the graph of services to start when the Faust worker for an app starts.
-
enable_kafka
= True¶
-
enable_kafka_producer
= None¶
-
enable_kafka_consumer
= None¶
-
enable_web
= None¶
-
enable_sensors
= True¶
-
-
class
faust.app.base.
App
(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = None, beacon: mode.utils.types.trees.NodeT = None, **options) → None[source]¶ Faust Application.
- Parameters
id (str) – Application ID.
- Keyword Arguments
loop (asyncio.AbstractEventLoop) – optional event loop to use.
See also
Application Parameters – for supported keyword arguments.
-
class
BootStrategy
(app: faust.types.app.AppT, *, enable_web: bool = None, enable_kafka: bool = None, enable_kafka_producer: bool = None, enable_kafka_consumer: bool = None, enable_sensors: bool = None) → None¶ App startup strategy.
The startup strategy defines the graph of services to start when the Faust worker for an app starts.
-
enable_kafka
= True¶
-
enable_kafka_consumer
= None¶
-
enable_kafka_producer
= None¶
-
enable_sensors
= True¶
-
enable_web
= None¶
-
-
class
Settings
(id: str, *, version: int = None, broker: Union[str, yarl.URL, List[yarl.URL]] = None, broker_client_id: str = None, broker_request_timeout: Union[datetime.timedelta, float, str] = None, broker_credentials: Union[faust.types.auth.CredentialsT, ssl.SSLContext] = None, broker_commit_every: int = None, broker_commit_interval: Union[datetime.timedelta, float, str] = None, broker_commit_livelock_soft_timeout: Union[datetime.timedelta, float, str] = None, broker_session_timeout: Union[datetime.timedelta, float, str] = None, broker_heartbeat_interval: Union[datetime.timedelta, float, str] = None, broker_check_crcs: bool = None, broker_max_poll_records: int = None, agent_supervisor: Union[_T, str] = None, store: Union[str, yarl.URL] = None, cache: Union[str, yarl.URL] = None, web: Union[str, yarl.URL] = None, web_enabled: bool = True, processing_guarantee: Union[str, faust.types.enums.ProcessingGuarantee] = None, timezone: datetime.tzinfo = None, autodiscover: Union[bool, Iterable[str], Callable[Iterable[str]]] = None, origin: str = None, canonical_url: Union[str, yarl.URL] = None, datadir: Union[pathlib.Path, str] = None, tabledir: Union[pathlib.Path, str] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, logging_config: Dict = None, loghandlers: List[logging.Handler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = None, topic_allow_declare: bool = None, id_format: str = None, reply_to: str = None, reply_to_prefix: str = None, reply_create_topic: bool = None, reply_expires: Union[datetime.timedelta, float, str] = None, ssl_context: ssl.SSLContext = None, stream_buffer_maxsize: int = None, stream_wait_empty: bool = None, stream_ack_cancelled_tasks: bool = None, stream_ack_exceptions: bool = None, stream_publish_on_commit: bool = None, stream_recovery_delay: Union[datetime.timedelta, float, str] = None, producer_linger_ms: int = None, producer_max_batch_size: int = None, producer_acks: int = None, producer_max_request_size: int = None, producer_compression_type: str = None, producer_partitioner: Union[_T, str] = None, producer_request_timeout: Union[datetime.timedelta, float, str] = None, producer_api_version: str = None, consumer_max_fetch_size: int = None, consumer_auto_offset_reset: str = None, web_bind: str = None, web_port: int = None, web_host: str = None, web_transport: Union[str, yarl.URL] = None, web_in_thread: bool = None, web_cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[_T, str] = None, ConsumerScheduler: Union[_T, str] = None, Stream: Union[_T, str] = None, Table: Union[_T, str] = None, SetTable: Union[_T, str] = None, TableManager: Union[_T, str] = None, Serializers: Union[_T, str] = None, Worker: Union[_T, str] = None, PartitionAssignor: Union[_T, str] = None, LeaderAssignor: Union[_T, str] = None, Router: Union[_T, str] = None, Topic: Union[_T, str] = None, HttpClient: Union[_T, str] = None, Monitor: Union[_T, str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None¶ -
-
LeaderAssignor
¶ - Return type
-
PartitionAssignor
¶ - Return type
-
TableManager
¶ - Return type
-
agent_supervisor
¶ - Return type
-
autodiscover
= False¶
-
broker_check_crcs
= True¶
-
broker_client_id
= 'faust-1.5.5'¶
-
broker_commit_every
= 10000¶
-
broker_credentials
¶ - Return type
-
consumer_auto_offset_reset
= 'earliest'¶
-
consumer_max_fetch_size
= 4194304¶
-
id_format
= '{id}-v{self.version}'¶
-
key_serializer
= 'raw'¶
-
logging_config
= None¶
-
processing_guarantee
¶ - Return type
-
producer_acks
= -1¶
-
producer_api_version
= 'auto'¶
-
producer_compression_type
= None¶
-
producer_linger_ms
= 0¶
-
producer_max_batch_size
= 16384¶
-
producer_max_request_size
= 1000000¶
-
producer_partitioner
¶
-
reply_create_topic
= False¶
-
reply_to_prefix
= 'f-reply-'¶
-
ssl_context
= None¶
-
stream_ack_cancelled_tasks
= True¶
-
stream_ack_exceptions
= True¶
-
stream_buffer_maxsize
= 4096¶
-
stream_publish_on_commit
= False¶
-
stream_wait_empty
= True¶
-
table_standby_replicas
= 1¶
-
timezone
= datetime.timezone.utc¶
-
topic_allow_declare
= True¶
-
topic_partitions
= 8¶
-
topic_replication_factor
= 1¶
-
value_serializer
= 'json'¶
-
web_bind
= '0.0.0.0'¶
-
web_cors_options
= None¶
-
web_host
= 'build-8929922-project-230058-faust'¶
-
web_in_thread
= False¶
-
web_port
= 6066¶
-
worker_redirect_stdouts
= True¶
-
worker_redirect_stdouts_level
= 'WARN'¶
-
-
client_only
= False¶ Set this to True if app should only start the services required to operate as an RPC client (producer and simple reply consumer).
-
producer_only
= False¶ Set this to True if app should run without consumer/tables.
-
tracer
= None¶ Optional tracing support.
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
config_from_object
(obj: Any, *, silent: bool = False, force: bool = False) → None[source]¶ Read configuration from object.
Object is either an actual object or the name of a module to import.
Examples
>>> app.config_from_object('myproj.faustconfig')
>>> from myproj import faustconfig >>> app.config_from_object(faustconfig)
-
discover
(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[Any] = [<built-in method search of _sre.SRE_Pattern object>, '.__main__']) → None[source]¶ - Return type
None
-
topic
(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, maxsize: int = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → faust.types.topics.TopicT[source]¶ Create topic description.
Topics are named channels (for example a Kafka topic), that exist on a server. To make an ephemeral local communication channel use:
channel()
.See also
- Return type
TopicT
[]
-
channel
(*, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, maxsize: int = None, loop: asyncio.events.AbstractEventLoop = None) → faust.types.channels.ChannelT[source]¶ Create new channel.
By default this will create an in-memory channel used for intra-process communication, but in practice channels can be backed by any transport (network or even means of inter-process communication).
See also
- Return type
ChannelT
[]
-
agent
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT][source]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
actor
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT]¶ Create Agent from async def function.
It can be a regular async function:
@app.agent() async def my_agent(stream): async for number in stream: print(f'Received: {number!r}')
Or it can be an async iterator that yields values. These values can be used as the reply in an RPC-style call, or for sinks: callbacks that forward events to other agents/topics/statsd, and so on:
@app.agent(sink=[log_topic]) async def my_agent(requests): async for number in requests: yield number * 2
-
task
(fun: Union[Callable[AppT, Awaitable], Callable[Awaitable]] = None, *, on_leader: bool = False, traced: bool = True) → Union[Callable[Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]], Union[Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]]], Callable[faust.types.app.AppT, Awaitable], Callable[Awaitable]][source]¶ Define an async def function to be started with the app.
This is like
timer()
but a one-shot task only executed at worker startup (after recovery and the worker is fully ready for operation).The function may take zero, or one argument. If the target function takes an argument, the
app
argument is passed:>>> @app.task >>> async def on_startup(app): ... print('STARTING UP: %r' % (app,))
Nullary functions are also supported:
>>> @app.task >>> async def on_startup(): ... print('STARTING UP')
-
timer
(interval: Union[datetime.timedelta, float, str], on_leader: bool = False, traced: bool = True, name: str = None, max_drift_correction: float = 0.1) → Callable[source]¶ Define an async def function to be run at periodic intervals.
Like
task()
, but executes periodically until the worker is shut down.This decorator takes an async function and adds it to a list of timers started with the app.
- Parameters
interval (Seconds) – How often the timer executes in seconds.
on_leader (bool) – Should the timer only run on the leader?
Example
>>> @app.timer(interval=10.0) >>> async def every_10_seconds(): ... print('TEN SECONDS JUST PASSED')
>>> app.timer(interval=5.0, on_leader=True) >>> async def every_5_seconds(): ... print('FIVE SECONDS JUST PASSED. ALSO, I AM THE LEADER!')
- Return type
-
crontab
(cron_format: str, *, timezone: datetime.tzinfo = None, on_leader: bool = False, traced: bool = True) → Callable[source]¶ Define periodic task using Crontab description.
This is an
async def
function to be run at the fixed times, defined by the Cron format.Like
timer()
, but executes at fixed times instead of executing at certain intervals.This decorator takes an async function and adds it to a list of Cronjobs started with the app.
- Parameters
cron_format (
str
) – The Cron spec defining fixed times to run the decorated function.- Keyword Arguments
timezone – The timezone to be taken into account for the Cron jobs. If not set value from
timezone
will be taken.on_leader – Should the Cron job only run on the leader?
Example
>>> @app.crontab(cron_format='30 18 * * *', timezone=pytz.timezone('US/Pacific')) >>> async def every_6_30_pm_pacific(): ... print('IT IS 6:30pm')
>>> app.crontab(cron_format='30 18 * * *', on_leader=True) >>> async def every_6_30_pm(): ... print('6:30pm UTC; ALSO, I AM THE LEADER!')
- Return type
-
service
(cls: Type[mode.types.services.ServiceT]) → Type[mode.types.services.ServiceT][source]¶ Decorate
mode.Service
to be started with the app.Examples
from mode import Service @app.service class Foo(Service): ...
-
stream
(channel: Union[AsyncIterable, Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]¶ Create new stream from channel/topic/iterable/async iterable.
- Parameters
channel (
Union
[AsyncIterable
[+T_co],Iterable
[+T_co]]) – Iterable to stream over (async or non-async).kwargs (
Any
) – SeeStream
.
- Return type
StreamT
[+T_co]- Returns
to iterate over events in the stream.
- Return type
-
Table
(name: str, *, default: Callable[Any] = None, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ Define new table.
- Parameters
name (
str
) – Name used for table, note that two tables living in the same application cannot have the same name.default (
Optional
[Callable
[[],Any
]]) – A callable, or type that will return a default value for keys missing in this table.window (
Optional
[WindowT
]) – A windowing strategy to wrap this window in.
Examples
>>> table = app.Table('user_to_amount', default=int) >>> table['George'] 0 >>> table['Elaine'] += 1 >>> table['Elaine'] += 1 >>> table['Elaine'] 2
- Return type
TableT
[~KT, ~VT]
-
SetTable
(name: str, *, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ - Return type
TableT
[~KT, ~VT]
-
page
(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, name: str = None) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Type[faust.web.views.View]][source]¶
-
table_route
(table: faust.types.tables.CollectionT, shard_param: str = None, *, query_param: str = None, match_info: str = None) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
command
(*options, base: Optional[Type[faust.app.base._AppCommand]] = None, **kwargs) → Callable[Callable, Type[faust.app.base._AppCommand]][source]¶
-
trace
(name: str, trace_enabled: bool = True, **extra_context) → ContextManager[source]¶ - Return type
ContextManager
[+T_co]
-
traced
(fun: Callable, name: str = None, sample_rate: float = 1.0, **context) → Callable[source]¶ - Return type
-
FlowControlQueue
(maxsize: int = None, *, clear_on_resume: bool = False, loop: asyncio.events.AbstractEventLoop = None) → mode.utils.queues.ThrowableQueue[source]¶ Like
asyncio.Queue
, but can be suspended/resumed.- Return type
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ Commit offset for acked messages in specified topics’.
Warning
This will commit acked messages in all topics if the topics argument is passed in as
None
.- Return type
-
logger
= <Logger faust.app.base (WARNING)>¶
-
coroutine
maybe_start_client
(self) → None[source]¶ Start the app in Client-Only mode if not started as Server.
- Return type
None
-
coroutine
on_first_start
(self) → None[source]¶ Service started for the first time in this process.
- Return type
None
-
coroutine
on_init_extra_service
(self, service: Union[mode.types.services.ServiceT, Type[mode.types.services.ServiceT]]) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
-
coroutine
send
(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send event to channel/topic.
- Parameters
channel (
Union
[ChannelT
[],str
]) – Channel/topic or the name of a topic to send event to.partition (
Optional
[int
]) – Specific partition to send to. If not set the partition will be chosen by the partitioner.timestamp (
Optional
[float
]) – Epoch seconds (from Jan 1 1970 UTC) to use as the message timestamp. Defaults to current time.headers (
Union
[List
[Tuple
[str
,bytes
]],Mapping
[str
,bytes
],None
]) – Mapping of key/value pairs, or iterable of key value pairs to use as headers for the message.key_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).value_serializer (
Union
[CodecT
,str
,None
]) – Serializer to use (if value is not model).callback (
Optional
[Callable
[[FutureMessage
[]],Union
[None
,Awaitable
[None
]]]]) –Called after the message is fully delivered to the channel, but not to the consumer. Signature must be unary as the
FutureMessage
future is passed to it.The resulting
faust.types.tuples.RecordMetadata
object is then available asfut.result()
.
- Return type
-
coroutine
start_client
(self) → None[source]¶ Start the app in Client-Only mode necessary for RPC requests.
Notes
Once started as a client the app cannot be restarted as Server.
- Return type
None
-
transport
¶ Message transport. :rtype:
TransportT
-
cache
¶ - Return type
-
topics
[source]¶ Topic Conductor.
This is the mediator that moves messages fetched by the Consumer into the streams.
It’s also a set of registered topics by string topic name, so you can check if a topic is being consumed from by doing
topic in app.topics
.
-
flow_control
[source]¶ Flow control of streams.
This object controls flow into stream queues, and can also clear all buffers.
-
http_client
¶ HTTP Client Session. :rtype:
ClientSession
faust.app.router
¶
Route messages to Faust nodes by partitioning.
-
class
faust.app.router.
Router
(app: faust.types.app.AppT) → None[source]¶ Router for
app.router
.-
key_store
(table_name: str, key: Union[bytes, faust.types.core._ModelT, Any, None]) → yarl.URL[source]¶ - Return type
-
table_metadata
(table_name: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]¶ - Return type
-
Agents¶
faust.agents
¶
-
class
faust.agents.
Agent
(fun: Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, isolated_partitions: bool = False, use_reply_headers: bool = None, **kwargs) → None[source]¶ Agent.
This is the type of object returned by the
@app.agent
decorator.-
supervisor
= None¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
clone
(*, cls: Type[faust.types.agents.AgentT] = None, **kwargs) → faust.types.agents.AgentT[source]¶ - Return type
AgentT
[]
-
test_context
(channel: faust.types.channels.ChannelT = None, supervisor_strategy: mode.types.supervisors.SupervisorStrategyT = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, **kwargs) → faust.types.agents.AgentTestWrapperT[source]¶ - Return type
-
actor_from_stream
(stream: Optional[faust.types.streams.StreamT], *, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, channel: faust.types.channels.ChannelT = None) → faust.types.agents.ActorT[Union[AsyncIterable, Awaitable]][source]¶ - Return type
ActorT
[]
-
add_sink
(sink: Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]) → None[source]¶ - Return type
None
-
stream
(channel: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
coroutine
ask
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]¶ - Return type
-
coroutine
ask_nowait
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]¶ - Return type
-
coroutine
cast
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → None[source]¶ - Return type
None
-
coroutine
join
(self, values: Union[AsyncIterable[Union[bytes, faust.types.core._ModelT, Any]], Iterable[Union[bytes, faust.types.core._ModelT, Any]]], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvjoin
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
kvmap
(items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]¶ - Return type
-
logger
= <Logger faust.agents.agent (WARNING)>¶
-
map
(values: Union[AsyncIterable, Iterable], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[source]¶ - Return type
AsyncIterator
[+T_co]
-
coroutine
on_isolated_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_isolated_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
- Return type
None
- Return type
None
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to topic used by agent.
- Return type
-
channel_iterator
¶ - Return type
AsyncIterator
[+T_co]
-
-
faust.agents.
AgentFun
¶ alias of
typing.Callable
-
class
faust.agents.
AgentT
(fun: Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], *, name: str = None, app: faust.types.agents._AppT = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, isolated_partitions: bool = False, **kwargs) → None[source]¶ -
test_context
(channel: faust.types.channels.ChannelT = None, supervisor_strategy: mode.types.supervisors.SupervisorStrategyT = None, **kwargs) → faust.types.agents.AgentTestWrapperT[source]¶ - Return type
-
add_sink
(sink: Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]) → None[source]¶ - Return type
None
-
clone
(*, cls: Type[AgentT] = None, **kwargs) → faust.types.agents.AgentT[source]¶ - Return type
AgentT
[]
-
channel_iterator
¶ - Return type
AsyncIterator
[+T_co]
-
coroutine
ask
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]¶ - Return type
-
coroutine
cast
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → None[source]¶ - Return type
None
-
coroutine
join
(self, values: Union[AsyncIterable[Union[bytes, faust.types.core._ModelT, Any]], Iterable[Union[bytes, faust.types.core._ModelT, Any]]], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvjoin
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvmap
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]¶
-
coroutine
map
(self, values: Union[AsyncIterable, Iterable], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[source]¶
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.agents.
AgentManager
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Agent manager.
-
logger
= <Logger faust.agents.manager (WARNING)>¶
-
-
class
faust.agents.
AgentManagerT
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶
-
class
faust.agents.
ReplyConsumer
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Consumer responsible for redelegation of replies received.
-
coroutine
add
(self, correlation_id: str, promise: faust.agents.replies.ReplyPromise) → None[source]¶ - Return type
None
-
logger
= <Logger faust.agents.replies (WARNING)>¶
-
coroutine
faust.agents.actor
¶
Actor - Individual Agent instances.
-
class
faust.agents.actor.
Actor
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ An actor is a specific agent instance.
-
mundane_level
= 'debug'¶
-
logger
= <Logger faust.agents.actor (WARNING)>¶
-
coroutine
on_isolated_partition_assigned
(self, tp: faust.types.tuples.TP) → None[source]¶ - Return type
None
-
-
class
faust.agents.actor.
AsyncIterableActor
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ Used for agent function that yields.
-
logger
= <Logger faust.agents.actor (WARNING)>¶
-
-
class
faust.agents.actor.
AwaitableActor
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ Used for actor function that do not yield.
-
logger
= <Logger faust.agents.actor (WARNING)>¶
-
faust.agents.agent
¶
Agent implementation.
-
class
faust.agents.agent.
Agent
(fun: Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, isolated_partitions: bool = False, use_reply_headers: bool = None, **kwargs) → None[source]¶ Agent.
This is the type of object returned by the
@app.agent
decorator.-
supervisor
= None¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
clone
(*, cls: Type[faust.types.agents.AgentT] = None, **kwargs) → faust.types.agents.AgentT[source]¶ - Return type
AgentT
[]
-
test_context
(channel: faust.types.channels.ChannelT = None, supervisor_strategy: mode.types.supervisors.SupervisorStrategyT = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, **kwargs) → faust.types.agents.AgentTestWrapperT[source]¶ - Return type
-
actor_from_stream
(stream: Optional[faust.types.streams.StreamT], *, index: int = None, active_partitions: Set[faust.types.tuples.TP] = None, channel: faust.types.channels.ChannelT = None) → faust.types.agents.ActorT[Union[AsyncIterable, Awaitable]][source]¶ - Return type
ActorT
[]
-
add_sink
(sink: Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]) → None[source]¶ - Return type
None
-
stream
(channel: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
coroutine
ask
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]¶ - Return type
-
coroutine
ask_nowait
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]¶ - Return type
-
coroutine
cast
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → None[source]¶ - Return type
None
-
coroutine
join
(self, values: Union[AsyncIterable[Union[bytes, faust.types.core._ModelT, Any]], Iterable[Union[bytes, faust.types.core._ModelT, Any]]], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvjoin
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
kvmap
(items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]¶ - Return type
-
logger
= <Logger faust.agents.agent (WARNING)>¶
-
map
(values: Union[AsyncIterable, Iterable], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[source]¶ - Return type
AsyncIterator
[+T_co]
-
coroutine
on_isolated_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_isolated_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
- Return type
None
- Return type
None
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ Send message to topic used by agent.
- Return type
-
channel_iterator
¶ - Return type
AsyncIterator
[+T_co]
-
faust.agents.manager
¶
Agent manager.
-
class
faust.agents.manager.
AgentManager
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Agent manager.
-
logger
= <Logger faust.agents.manager (WARNING)>¶
-
faust.agents.models
¶
Models used by agents internally.
-
class
faust.agents.models.
ReqRepRequest
(value, reply_to, correlation_id, *, __strict__=True, __faust=None, **kwargs) → None[source]¶ Value wrapped in a Request-Reply request.
-
value
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
reply_to
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
correlation_id
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
asdict
()¶
-
-
class
faust.agents.models.
ReqRepResponse
(key, value, correlation_id, *, __strict__=True, __faust=None, **kwargs) → None[source]¶ Request-Reply response.
-
key
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
value
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
correlation_id
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
asdict
()¶
-
faust.agents.replies
¶
Agent replies: waiting for replies, sending them, etc.
-
class
faust.agents.replies.
ReplyPromise
(reply_to: str, correlation_id: str, **kwargs) → None[source]¶ Reply promise can be
await
-ed to wait until result ready.
-
class
faust.agents.replies.
BarrierState
(reply_to: str, **kwargs) → None[source]¶ State of pending/complete barrier.
A barrier is a synchronization primitive that will wait until a group of coroutines have completed.
-
size
= 0¶ This is the size while the messages are being sent. (it’s a tentative total, added to until the total is finalized).
-
total
= 0¶ This is the actual total when all messages have been sent. It’s set by
finalize()
.
-
fulfilled
= 0¶ The number of results we have received.
-
pending
= None¶ Set of pending replies that this barrier is composed of.
-
get_nowait
() → faust.agents.replies.ReplyTuple[source]¶ Return next reply, or raise
asyncio.QueueEmpty
.- Return type
ReplyTuple
-
iterate
() → AsyncIterator[faust.agents.replies.ReplyTuple][source]¶ Iterate over results as arrive.
- Return type
AsyncIterator
[ReplyTuple
]
-
-
class
faust.agents.replies.
ReplyConsumer
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Consumer responsible for redelegation of replies received.
-
coroutine
add
(self, correlation_id: str, promise: faust.agents.replies.ReplyPromise) → None[source]¶ - Return type
None
-
logger
= <Logger faust.agents.replies (WARNING)>¶
-
coroutine
Fixups¶
faust.fixups
¶
Transport registry.
faust.fixups.base
¶
Fixups - Base implementation.
faust.fixups.django
¶
Django Fixups - Integration with Django.
-
class
faust.fixups.django.
Fixup
(app: faust.types.app.AppT) → None[source]¶ Django fixup.
This fixup is enabled if
the
DJANGO_SETTINGS_MODULE
environment variable is set,the django package is installed.
Once enabled it will modify the following features:
Autodiscovery
If
faust.App(autodiscovery=True)
, the Django fixup will automatically autodiscover agents/tasks/web views, and so on found in installed Django apps.Setup
The Django machinery will be set up when Faust commands are executed.
Models¶
faust.models.base
¶
Model descriptions.
The model describes the components of a data structure, kind of like a struct in C, but there’s no limitation of what type of data structure the model is, or what it’s used for.
A record (faust.models.record) is a model type that serialize into dictionaries, so the model describe the fields, and their types:
>>> class Point(Record):
... x: int
... y: int
>>> p = Point(10, 3)
>>> assert p.x == 10
>>> assert p.y == 3
>>> p
<Point: x=10, y=3>
>>> payload = p.dumps(serializer='json')
'{"x": 10, "y": 3, "__faust": {"ns": "__main__.Point"}}'
>>> p2 = Record.loads(payload)
>>> p2
<Point: x=10, y=3>
Models are mainly used for describing the data in messages: both keys and values can be described as models.
-
faust.models.base.
registry
= {'@ClientAssignment': <class 'faust.assignor.client_assignment.ClientAssignment'>, '@ClientMetadata': <class 'faust.assignor.client_assignment.ClientMetadata'>, '@ClusterAssignment': <class 'faust.assignor.cluster_assignment.ClusterAssignment'>, '@ReqRepRequest': <class 'faust.agents.models.ReqRepRequest'>, '@ReqRepResponse': <class 'faust.agents.models.ReqRepResponse'>, 'faust.agents.models.ModelReqRepRequest': <class 'faust.agents.models.ModelReqRepRequest'>, 'faust.agents.models.ModelReqRepResponse': <class 'faust.agents.models.ModelReqRepResponse'>}¶ Global map of namespace -> Model, used to find model classes by name. Every single model defined is added here automatically when a model class is defined.
-
class
faust.models.base.
Model
(*args, **kwargs) → None[source]¶ Meta description model for serialization.
-
classmethod
loads
(s: bytes, *, default_serializer: Union[faust.types.codecs.CodecT, str, None] = None, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → faust.types.models.ModelT[source]¶ Deserialize model object from bytes.
- Keyword Arguments
serializer (CodecArg) – Default serializer to use if no custom serializer was set for this model subclass.
- Return type
-
classmethod
-
class
faust.models.base.
FieldDescriptor
(field: str, type: Type, model: Type[faust.types.models.ModelT], required: bool = True, default: Any = None, parent: faust.types.models.FieldDescriptorT = None) → None[source]¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
- Parameters
-
field
= None¶ Name of attribute on Model.
-
type
= None¶ Type of value (e.g.
int
, orOptional[int]
)).
-
model
= None¶ The model class this field is associated with.
-
required
= True¶ Set if a value for this field is required (cannot be
None
).
-
default
= None¶ Default value for non-required field.
faust.models.record
¶
Record - Dictionary Model.
-
class
faust.models.record.
Record
→ None[source]¶ Describes a model type that is a record (Mapping).
Examples
>>> class LogEvent(Record, serializer='json'): ... severity: str ... message: str ... timestamp: float ... optional_field: str = 'default value'
>>> event = LogEvent( ... severity='error', ... message='Broken pact', ... timestamp=666.0, ... )
>>> event.severity 'error'
>>> serialized = event.dumps() '{"severity": "error", "message": "Broken pact", "timestamp": 666.0}'
>>> restored = LogEvent.loads(serialized) <LogEvent: severity='error', message='Broken pact', timestamp=666.0>
>>> # You can also subclass a Record to create a new record >>> # with additional fields >>> class RemoteLogEvent(LogEvent): ... url: str
>>> # You can also refer to record fields and pass them around: >>> LogEvent.severity >>> <FieldDescriptor: LogEvent.severity (str)>
-
classmethod
from_data
(data: Mapping, *, preferred_type: Type[faust.types.models.ModelT] = None) → faust.models.record.Record[source]¶ - Return type
-
classmethod
Sensors¶
faust.sensors
¶
-
class
faust.sensors.
Monitor
(*, max_avg_history: int = None, max_commit_latency_history: int = None, max_send_latency_history: int = None, max_assignment_latency_history: int = None, messages_sent: int = 0, tables: MutableMapping[str, faust.sensors.monitor.TableState] = None, messages_active: int = 0, events_active: int = 0, messages_received_total: int = 0, messages_received_by_topic: Counter[str] = None, events_total: int = 0, events_by_stream: Counter[faust.types.streams.StreamT] = None, events_by_task: Counter[_asyncio.Task] = None, events_runtime: Deque[float] = None, commit_latency: Deque[float] = None, send_latency: Deque[float] = None, assignment_latency: Deque[float] = None, events_s: int = 0, messages_s: int = 0, events_runtime_avg: float = 0.0, topic_buffer_full: Counter[faust.types.topics.TopicT] = None, rebalances: int = None, rebalance_return_latency: Deque[float] = None, rebalance_end_latency: Deque[float] = None, rebalance_return_avg: float = 0.0, rebalance_end_avg: float = 0.0, time: Callable[float] = <built-in function monotonic>, **kwargs) → None[source]¶ Default Faust Sensor.
This is the default sensor, recording statistics about events, etc.
-
send_errors
= 0¶ Number of produce operations that ended in error.
-
assignments_completed
= 0¶ Number of partition assignments completed.
-
assignments_failed
= 0¶ Number of partitions assignments that failed.
-
max_avg_history
= 100¶ Max number of total run time values to keep to build average.
-
max_commit_latency_history
= 30¶ Max number of commit latency numbers to keep.
-
max_send_latency_history
= 30¶ Max number of send latency numbers to keep.
-
max_assignment_latency_history
= 30¶ Max number of assignment latency numbers to keep.
-
rebalances
= 0¶ Number of rebalances seen by this worker.
-
tables
= None¶ Mapping of tables
-
commit_latency
= None¶ Deque of commit latency values
-
send_latency
= None¶ Deque of send latency values
-
assignment_latency
= None¶ Deque of assignment latency values.
-
rebalance_return_latency
= None¶
-
rebalance_end_latency
= None¶
-
messages_active
= 0¶ Number of messages currently being processed.
-
messages_received_total
= 0¶ Number of messages processed in total.
-
messages_received_by_topic
= None¶ Count of messages received by topic
-
messages_sent
= 0¶ Number of messages sent in total.
-
messages_sent_by_topic
= None¶ Number of messages sent by topic.
-
messages_s
= 0¶ Number of messages being processed this second.
-
events_active
= 0¶ Number of events currently being processed.
-
events_total
= 0¶ Number of events processed in total.
-
events_by_task
= None¶ Count of events processed by task
-
events_by_stream
= None¶ Count of events processed by stream
-
events_s
= 0¶ Number of events being processed this second.
-
events_runtime_avg
= 0.0¶ Average event runtime over the last second.
-
events_runtime
= None¶ Deque of run times used for averages
-
topic_buffer_full
= None¶ Counter of times a topics buffer was full
-
metric_counts
= None¶ Arbitrary counts added by apps
-
tp_committed_offsets
= None¶ Last committed offsets by TopicPartition
-
tp_read_offsets
= None¶ Last read offsets by TopicPartition
-
tp_end_offsets
= None¶ Log end offsets by TopicPartition
-
secs_since
(start_time: float) → float[source]¶ Given timestamp start, return number of seconds since that time.
- Return type
-
ms_since
(start_time: float) → float[source]¶ Given timestamp start, return number of ms since that time.
- Return type
-
logger
= <Logger faust.sensors.monitor (WARNING)>¶
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_tp_commit
(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
-
class
faust.sensors.
Sensor
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Base class for sensors.
This sensor does not do anything at all, but can be subclassed to create new monitors.
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Message sent to a stream as an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Event was acknowledged by stream.
Notes
Acknowledged means a stream finished processing the event, but given that multiple streams may be handling the same event, the message cannot be committed before all streams have processed it. When all streams have acknowledged the event, it will go through
on_message_out()
just before offsets are committed.- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
on_rebalance_return
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Consumer replied assignment is done to broker.
- Return type
None
-
on_rebalance_end
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Cluster rebalance fully completed (including recovery).
- Return type
None
-
logger
= <Logger faust.sensors.base (WARNING)>¶
-
-
class
faust.sensors.
SensorDelegate
(app: faust.types.app.AppT) → None[source]¶ A class that delegates sensor methods to a list of sensors.
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ - Return type
None
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ - Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ - Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ - Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
-
class
faust.sensors.
TableState
(table: faust.types.tables.CollectionT, *, keys_retrieved: int = 0, keys_updated: int = 0, keys_deleted: int = 0) → None[source]¶ Represents the current state of a table.
-
table
= None¶
-
keys_retrieved
= 0¶ Number of times a key has been retrieved from this table.
-
keys_updated
= 0¶ Number of times a key has been created/changed in this table.
-
keys_deleted
= 0¶ Number of times a key has been deleted from this table.
-
faust.sensors.base
¶
Base-interface for sensors.
-
class
faust.sensors.base.
Sensor
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Base class for sensors.
This sensor does not do anything at all, but can be subclassed to create new monitors.
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Message sent to a stream as an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Event was acknowledged by stream.
Notes
Acknowledged means a stream finished processing the event, but given that multiple streams may be handling the same event, the message cannot be committed before all streams have processed it. When all streams have acknowledged the event, it will go through
on_message_out()
just before offsets are committed.- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
on_rebalance_return
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Consumer replied assignment is done to broker.
- Return type
None
-
on_rebalance_end
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Cluster rebalance fully completed (including recovery).
- Return type
None
-
logger
= <Logger faust.sensors.base (WARNING)>¶
-
-
class
faust.sensors.base.
SensorDelegate
(app: faust.types.app.AppT) → None[source]¶ A class that delegates sensor methods to a list of sensors.
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ - Return type
None
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ - Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ - Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ - Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
faust.sensors.datadog
¶
Monitor using datadog.
-
class
faust.sensors.datadog.
DatadogMonitor
(host: str = 'localhost', port: int = 8125, prefix: str = 'faust-app', rate: float = 1.0, **kwargs) → None[source]¶ Datadog Faust Sensor.
This sensor, records statistics to datadog agents along with computing metrics for the stats server
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
on_rebalance_return
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Consumer replied assignment is done to broker.
- Return type
None
-
on_rebalance_end
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Cluster rebalance fully completed (including recovery).
- Return type
None
-
on_tp_commit
(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]¶ - Return type
None
-
logger
= <Logger faust.sensors.datadog (WARNING)>¶
-
faust.sensors.monitor
¶
Monitor - sensor tracking metrics.
-
class
faust.sensors.monitor.
TableState
(table: faust.types.tables.CollectionT, *, keys_retrieved: int = 0, keys_updated: int = 0, keys_deleted: int = 0) → None[source]¶ Represents the current state of a table.
-
table
= None¶
-
keys_retrieved
= 0¶ Number of times a key has been retrieved from this table.
-
keys_updated
= 0¶ Number of times a key has been created/changed in this table.
-
keys_deleted
= 0¶ Number of times a key has been deleted from this table.
-
-
class
faust.sensors.monitor.
Monitor
(*, max_avg_history: int = None, max_commit_latency_history: int = None, max_send_latency_history: int = None, max_assignment_latency_history: int = None, messages_sent: int = 0, tables: MutableMapping[str, faust.sensors.monitor.TableState] = None, messages_active: int = 0, events_active: int = 0, messages_received_total: int = 0, messages_received_by_topic: Counter[str] = None, events_total: int = 0, events_by_stream: Counter[faust.types.streams.StreamT] = None, events_by_task: Counter[_asyncio.Task] = None, events_runtime: Deque[float] = None, commit_latency: Deque[float] = None, send_latency: Deque[float] = None, assignment_latency: Deque[float] = None, events_s: int = 0, messages_s: int = 0, events_runtime_avg: float = 0.0, topic_buffer_full: Counter[faust.types.topics.TopicT] = None, rebalances: int = None, rebalance_return_latency: Deque[float] = None, rebalance_end_latency: Deque[float] = None, rebalance_return_avg: float = 0.0, rebalance_end_avg: float = 0.0, time: Callable[float] = <built-in function monotonic>, **kwargs) → None[source]¶ Default Faust Sensor.
This is the default sensor, recording statistics about events, etc.
-
send_errors
= 0¶ Number of produce operations that ended in error.
-
assignments_completed
= 0¶ Number of partition assignments completed.
-
assignments_failed
= 0¶ Number of partitions assignments that failed.
-
max_avg_history
= 100¶ Max number of total run time values to keep to build average.
-
max_commit_latency_history
= 30¶ Max number of commit latency numbers to keep.
-
max_send_latency_history
= 30¶ Max number of send latency numbers to keep.
-
max_assignment_latency_history
= 30¶ Max number of assignment latency numbers to keep.
-
rebalances
= 0¶ Number of rebalances seen by this worker.
-
tables
= None¶ Mapping of tables
-
commit_latency
= None¶ Deque of commit latency values
-
send_latency
= None¶ Deque of send latency values
-
assignment_latency
= None¶ Deque of assignment latency values.
-
rebalance_return_latency
= None¶
-
rebalance_end_latency
= None¶
-
messages_active
= 0¶ Number of messages currently being processed.
-
messages_received_total
= 0¶ Number of messages processed in total.
-
messages_received_by_topic
= None¶ Count of messages received by topic
-
messages_sent
= 0¶ Number of messages sent in total.
-
messages_sent_by_topic
= None¶ Number of messages sent by topic.
-
messages_s
= 0¶ Number of messages being processed this second.
-
events_active
= 0¶ Number of events currently being processed.
-
events_total
= 0¶ Number of events processed in total.
-
events_by_task
= None¶ Count of events processed by task
-
events_by_stream
= None¶ Count of events processed by stream
-
events_s
= 0¶ Number of events being processed this second.
-
events_runtime_avg
= 0.0¶ Average event runtime over the last second.
-
events_runtime
= None¶ Deque of run times used for averages
-
topic_buffer_full
= None¶ Counter of times a topics buffer was full
-
metric_counts
= None¶ Arbitrary counts added by apps
-
tp_committed_offsets
= None¶ Last committed offsets by TopicPartition
-
tp_read_offsets
= None¶ Last read offsets by TopicPartition
-
tp_end_offsets
= None¶ Log end offsets by TopicPartition
-
secs_since
(start_time: float) → float[source]¶ Given timestamp start, return number of seconds since that time.
- Return type
-
ms_since
(start_time: float) → float[source]¶ Given timestamp start, return number of ms since that time.
- Return type
-
logger
= <Logger faust.sensors.monitor (WARNING)>¶
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_topic_buffer_full
(topic: faust.types.topics.TopicT) → None[source]¶ Topic buffer full so conductor had to wait.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_initiated
(consumer: faust.types.transports.ConsumerT) → Any[source]¶ Consumer is about to commit topic offset.
- Return type
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_tp_commit
(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ Partition assignor is starting to assign partitions.
- Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
faust.sensors.statsd
¶
Monitor using Statsd.
-
class
faust.sensors.statsd.
StatsdMonitor
(host: str = 'localhost', port: int = 8125, prefix: str = 'faust-app', rate: float = 1.0, **kwargs) → None[source]¶ Statsd Faust Sensor.
This sensor, records statistics to Statsd along with computing metrics for the stats server
-
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ Message received by a consumer.
- Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶ Call when stream starts processing an event.
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ Call when stream is done processing an event.
- Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ All streams finished processing message.
- Return type
None
-
on_table_get
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key retrieved from table.
- Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ Value set for key in table.
- Return type
None
-
on_table_del
(table: faust.types.tables.CollectionT, key: Any) → None[source]¶ Key deleted from table.
- Return type
None
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ Consumer finished committing topic offset.
- Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ About to send a message.
- Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ Message successfully sent.
- Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ Error while sending message.
- Return type
None
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
on_assignment_completed
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict) → None[source]¶ Partition assignor completed assignment.
- Return type
None
-
on_rebalance_start
(app: faust.types.app.AppT) → Dict[source]¶ Cluster rebalance in progress.
- Return type
Dict
[~KT, ~VT]
-
on_rebalance_return
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Consumer replied assignment is done to broker.
- Return type
None
-
on_rebalance_end
(app: faust.types.app.AppT, state: Dict) → None[source]¶ Cluster rebalance fully completed (including recovery).
- Return type
None
-
on_tp_commit
(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]¶ - Return type
None
-
logger
= <Logger faust.sensors.statsd (WARNING)>¶
-
Serializers¶
faust.serializers.codecs
¶
Serialization utilities.
Supported codecs¶
raw - No encoding/serialization (bytes only).
json - json with UTF-8 encoding.
pickle - pickle with base64 encoding (not urlsafe).
binary - base64 encoding (not urlsafe).
Serialization by name¶
The dumps()
function takes a codec name and the object to encode,
then returns bytes:
>>> s = dumps('json', obj)
For the reverse direction, the loads()
function takes a codec
name and bytes to decode:
>>> obj = loads('json', s)
You can also combine encoders in the name, like in this case where json is combined with gzip compression:
>>> obj = loads('json|gzip', s)
Codec registry¶
Codecs are configured by name and this module maintains
a mapping from name to Codec
instance: the codecs
attribute.
You can add a new codec to this mapping by:
>>> from faust.serializers import codecs
>>> codecs.register(custom, custom_serializer())
A codec subclass requires two methods to be implemented: _loads()
and _dumps()
:
import msgpack
from faust.serializers import codecs
class raw_msgpack(codecs.Codec):
def _dumps(self, obj: Any) -> bytes:
return msgpack.dumps(obj)
def _loads(self, s: bytes) -> Any:
return msgpack.loads(s)
Our codec now encodes/decodes to raw msgpack format, but we may also need to transfer this payload over a transport easily confused by binary data, such as JSON where everything is Unicode.
You can chain codecs together, so to add a binary text encoding like Base64,
to your codec, we use the |
operator to form a combined codec:
def msgpack() -> codecs.Codec:
return raw_msgpack() | codecs.binary()
codecs.register('msgpack', msgpack())
At this point we monkey-patched Faust to support our codec, and we can use it to define records like this:
>>> from faust.serializers import Record
>>> class Point(Record, serializer='msgpack'):
... x: int
... y: int
The problem with monkey-patching is that we must make sure the patching happens before we use the feature.
Faust also supports registering codec extensions using setuptools entry points, so instead we can create an installable msgpack extension.
To do so we need to define a package with the following directory layout:
faust-msgpack/
setup.py
faust_msgpack.py
The first file, faust-msgpack/setup.py
, defines metadata about our
package and should look like the following example:
from setuptools import setup, find_packages
setup(
name='faust-msgpack',
version='1.0.0',
description='Faust msgpack serialization support',
author='Ola A. Normann',
author_email='ola@normann.no',
url='http://github.com/example/faust-msgpack',
platforms=['any'],
license='BSD',
packages=find_packages(exclude=['ez_setup', 'tests', 'tests.*']),
zip_safe=False,
install_requires=['msgpack-python'],
tests_require=[],
entry_points={
'faust.codecs': [
'msgpack = faust_msgpack:msgpack',
],
},
)
The most important part being the entry_points
key which tells
Faust how to load our plugin. We have set the name of our
codec to msgpack
and the path to the codec class
to be faust_msgpack:msgpack
. This will be imported by Faust
as from faust_msgpack import msgpack
, so we need to define
that part next in our faust-msgpack/faust_msgpack.py
module:
from faust.serializers import codecs
class raw_msgpack(codecs.Codec):
def _dumps(self, obj: Any) -> bytes:
return msgpack.dumps(s)
def msgpack() -> codecs.Codec:
return raw_msgpack() | codecs.binary()
That’s it! To install and use our new extension we do:
$ python setup.py install
At this point may want to publish this on PyPI to share the extension with other Faust users.
-
class
faust.serializers.codecs.
Codec
(children: Tuple[faust.types.codecs.CodecT, ...] = None, **kwargs) → None[source]¶ Base class for codecs.
-
children
= None¶ next steps in the recursive codec chain.
x = pickle | binary
returns codec with children set to(pickle, binary)
.
-
nodes
= None¶ cached version of children including this codec as the first node. could use chain below, but seems premature so just copying the list.
-
-
faust.serializers.codecs.
register
(name: str, codec: faust.types.codecs.CodecT) → None[source]¶ Register new codec in the codec registry.
- Return type
None
-
faust.serializers.codecs.
get_codec
(name_or_codec: Union[faust.types.codecs.CodecT, str, None]) → faust.types.codecs.CodecT[source]¶ Get codec by name.
- Return type
faust.serializers.registry
¶
Registry of supported codecs (serializers, compressors, etc.).
-
class
faust.serializers.registry.
Registry
(key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = 'json') → None[source]¶ Serializing message keys/values.
- Parameters
-
loads_key
(typ: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str], None], key: Optional[bytes], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → Union[bytes, faust.types.core._ModelT, Any, None][source]¶ Deserialize message key.
- Parameters
- Return type
-
loads_value
(typ: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str], None], value: Optional[bytes], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → Any[source]¶ Deserialize value.
-
dumps_key
(typ: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str], None], key: Union[bytes, faust.types.core._ModelT, Any, None], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None, skip: Tuple[Type, ...] = (<class 'bytes'>,)) → Optional[bytes][source]¶ Serialize key.
- Parameters
typ (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Model hint (can also bestr
/bytes
). Whentyp=str
orbytes
, raw serializer is assumed.key (
Union
[bytes
,_ModelT
,Any
,None
]) – The key value to serializer.serializer (
Union
[CodecT
,str
,None
]) – Codec to use for this key, if it is not a model type. If not set the default will be used (key_serializer
).
- Return type
-
dumps_value
(typ: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str], None], value: Union[bytes, faust.types.core._ModelT, Any], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None, skip: Tuple[Type, ...] = (<class 'bytes'>,)) → Optional[bytes][source]¶ Serialize value.
- Parameters
typ (
Union
[Type
[ModelT
],Type
[bytes
],Type
[str
],None
]) – Model hint (can also bestr
/bytes
). Whentyp=str
orbytes
, raw serializer is assumed.key – The value to serializer.
serializer (
Union
[CodecT
,str
,None
]) – Codec to use for this value, if it is not a model type. If not set the default will be used (value_serializer
).
- Return type
Stores¶
faust.stores
¶
Storage registry.
faust.stores.base
¶
Base class for table storage drivers.
-
class
faust.stores.base.
Store
(url: Union[str, yarl.URL], app: faust.types.app.AppT, table: faust.types.tables.CollectionT, *, table_name: str = '', key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Base class for table storage drivers.
-
logger
= <Logger faust.stores.base (WARNING)>¶
-
-
class
faust.stores.base.
SerializedStore
(url: Union[str, yarl.URL], app: faust.types.app.AppT, table: faust.types.tables.CollectionT, *, table_name: str = '', key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Base class for table storage drivers requiring serialization.
-
apply_changelog_batch
(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, KT], to_value: Callable[Any, VT]) → None[source]¶ - Return type
None
-
logger
= <Logger faust.stores.base (WARNING)>¶
-
faust.stores.memory
¶
In-memory table storage.
-
class
faust.stores.memory.
Store
(url: Union[str, yarl.URL], app: faust.types.app.AppT, table: faust.types.tables.CollectionT, *, table_name: str = '', key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Table storage using an in-memory dictionary.
-
apply_changelog_batch
(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, Any], to_value: Callable[Any, Any]) → None[source]¶ - Return type
None
-
logger
= <Logger faust.stores.memory (WARNING)>¶
-
faust.stores.rocksdb
¶
RocksDB storage.
-
class
faust.stores.rocksdb.
PartitionDB
(*args, **kwargs)[source]¶ Tuple of
(partition, rocksdb.DB)
.-
partition
¶ Alias for field number 0
-
db
¶ Alias for field number 1
-
-
class
faust.stores.rocksdb.
RocksDBOptions
(max_open_files: int = None, write_buffer_size: int = None, max_write_buffer_number: int = None, target_file_size_base: int = None, block_cache_size: int = None, block_cache_compressed_size: int = None, bloom_filter_size: int = None, **kwargs) → None[source]¶ Options required to open a RocksDB database.
-
max_open_files
= 943719¶
-
write_buffer_size
= 67108864¶
-
max_write_buffer_number
= 3¶
-
target_file_size_base
= 67108864¶
-
block_cache_size
= 2147483648¶
-
block_cache_compressed_size
= 524288000¶
-
bloom_filter_size
= 3¶
-
-
class
faust.stores.rocksdb.
Store
(url: Union[str, yarl.URL], app: faust.types.app.AppT, table: faust.types.tables.CollectionT, *, key_index_size: int = 10000, options: Mapping = None, **kwargs) → None[source]¶ RocksDB table storage.
-
offset_key
= b'__faust\x00offset__'¶
-
options
= None¶ Used to configure the RocksDB settings for table stores.
-
key_index_size
= None¶ Decides the size of the K=>TopicPartition index (10_000).
-
apply_changelog_batch
(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, Any], to_value: Callable[Any, Any]) → None[source]¶ - Return type
None
-
revoke_partitions
(table: faust.types.tables.CollectionT, tps: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
assign_partitions
(self, table: faust.types.tables.CollectionT, tps: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
logger
= <Logger faust.stores.rocksdb (WARNING)>¶
-
Tables¶
faust.tables
¶
-
class
faust.tables.
Collection
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Base class for changelog-backed data structures stored in Kafka.
-
data
¶
-
on_recover
(fun: Callable[Awaitable[None]]) → Callable[Awaitable[None]][source]¶ Add function as callback to be called on table recovery.
-
logger
= <Logger faust.tables.base (WARNING)>¶
-
coroutine
on_changelog_event
(self, event: faust.types.events.EventT) → None[source]¶ - Return type
None
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_recovery_completed
(self, active_tps: Set[faust.types.tuples.TP], standby_tps: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
-
class
faust.tables.
CollectionT
(app: faust.types.tables._AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: faust.types.tables._ModelArg = None, value_type: faust.types.tables._ModelArg = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ -
-
coroutine
on_changelog_event
(self, event: faust.types.events.EventT) → None[source]¶ - Return type
None
-
coroutine
-
class
faust.tables.
TableManager
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Manage tables used by Faust worker.
-
persist_offset_on_commit
(store: faust.types.stores.StoreT, tp: faust.types.tuples.TP, offset: int) → None[source]¶ Mark the persisted offset for a TP to be saved on commit.
This is used for “exactly_once” processing guarantee. Instead of writing the persisted offset to RocksDB when the message is sent, we write it to disk when the offset is committed.
- Return type
None
-
changelog_queue
¶ - Return type
-
logger
= <Logger faust.tables.manager (WARNING)>¶
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
-
class
faust.tables.
TableManagerT
(app: faust.types.tables._AppT, **kwargs) → None[source]¶
-
class
faust.tables.
Table
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Table (non-windowed).
-
class
WindowWrapper
(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None] = None, key_index: bool = False, key_index_table: faust.types.tables.TableT = None) → None¶ Windowed table wrapper.
A windowed table does not return concrete values when keys are accessed, instead
WindowSet
is returned so that the values can be further reduced to the wanted time period.-
ValueType
¶ alias of
WindowSet
-
clone
(relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
get_relative_timestamp
¶
-
key_index
= False¶
-
key_index_table
= None¶
-
on_del_key
(key: Any) → None¶ - Return type
None
-
on_recover
(fun: Callable[Awaitable[None]]) → Callable[Awaitable[None]]¶
-
on_set_key
(key: Any, value: Any) → None¶ - Return type
None
-
relative_to
(ts: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_field
(field: faust.types.models.FieldDescriptorT) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_now
() → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_stream
() → faust.types.tables.WindowWrapperT¶ - Return type
-
values
(event: faust.types.events.EventT = None) → ValuesView¶ - Return type
ValuesView
[+VT_co]
-
-
using_window
(window: faust.types.windows.WindowT, *, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
hopping
(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
tumbling
(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
on_key_set
(key: KT, value: VT) → None[source]¶ Handle that value for a key is being set.
- Return type
None
-
logger
= <Logger faust.tables.table (WARNING)>¶
-
class
-
class
faust.tables.
TableT
(app: faust.types.tables._AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: faust.types.tables._ModelArg = None, value_type: faust.types.tables._ModelArg = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ -
using_window
(window: faust.types.windows.WindowT, *, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
hopping
(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
faust.tables.base
¶
Base class Collection for Table and future data structures.
-
class
faust.tables.base.
Collection
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Base class for changelog-backed data structures stored in Kafka.
-
data
¶
-
on_recover
(fun: Callable[Awaitable[None]]) → Callable[Awaitable[None]][source]¶ Add function as callback to be called on table recovery.
-
logger
= <Logger faust.tables.base (WARNING)>¶
-
coroutine
on_changelog_event
(self, event: faust.types.events.EventT) → None[source]¶ - Return type
None
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_recovery_completed
(self, active_tps: Set[faust.types.tuples.TP], standby_tps: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
faust.tables.manager
¶
Tables (changelog stream).
-
class
faust.tables.manager.
TableManager
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Manage tables used by Faust worker.
-
persist_offset_on_commit
(store: faust.types.stores.StoreT, tp: faust.types.tuples.TP, offset: int) → None[source]¶ Mark the persisted offset for a TP to be saved on commit.
This is used for “exactly_once” processing guarantee. Instead of writing the persisted offset to RocksDB when the message is sent, we write it to disk when the offset is committed.
- Return type
None
-
changelog_queue
¶ - Return type
-
logger
= <Logger faust.tables.manager (WARNING)>¶
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
faust.tables.objects
¶
Storing objects in tables.
This is also used to store data structures such as sets/lists.
-
class
faust.tables.objects.
ChangeloggedObject
(manager: faust.tables.objects.ChangeloggedObjectManager, key: Any) → None[source]¶ A changelogged object in a
ChangeloggedObjectManager
store.
-
class
faust.tables.objects.
ChangeloggedObjectManager
(table: faust.tables.table.Table, **kwargs) → None[source]¶ Store of changelogged objects.
-
logger
= <Logger faust.tables.objects (WARNING)>¶
-
coroutine
on_rebalance
(self, table: faust.types.tables.CollectionT, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
faust.tables.recovery
¶
Table recovery after rebalancing.
-
exception
faust.tables.recovery.
RebalanceAgain
[source]¶ During rebalance, another rebalance happened.
-
class
faust.tables.recovery.
Recovery
(app: faust.types.app.AppT, tables: faust.types.tables.TableManagerT, **kwargs) → None[source]¶ Service responsible for recovering tables from changelog topics.
-
stats_interval
= 5.0¶
-
in_recovery
= False¶
-
standbys_pending
= False¶
-
signal_recovery_start
¶ - Return type
Event
-
signal_recovery_end
¶ - Return type
Event
-
signal_recovery_reset
¶ - Return type
Event
-
add_active
(table: faust.types.tables.CollectionT, tp: faust.types.tuples.TP) → None[source]¶ - Return type
None
-
add_standby
(table: faust.types.tables.CollectionT, tp: faust.types.tuples.TP) → None[source]¶ - Return type
None
-
standby_highwaters
= None¶ Standby highwaters by topic partition.
-
active_highwaters
= None¶ Active highwaters by topic partition.
-
logger
= <Logger faust.tables.recovery (WARNING)>¶
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
highwaters
= None¶ Mapping of highwaters by topic partition.
-
tp_to_table
= None¶ Mapping from topic partition to table
-
active_tps
= None¶ Set of active topic partitions.
-
standby_tps
= None¶ Set of standby topic partitions.
-
active_offsets
= None¶ Active offset by topic partition.
-
standby_offsets
= None¶ Standby offset by topic partition.
-
buffers
= None¶ Changelog event buffers by table. These are filled by background task _slurp_changelog, and need to be flushed before starting new recovery/stopping.
-
buffer_sizes
= None¶ Cache of buffer size by topic partition..
-
faust.tables.sets
¶
Storing sets in tables.
-
class
faust.tables.sets.
SetTable
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Table that maintains a dictionary of sets.
-
WindowWrapper
¶ alias of
SetWindowWrapper
-
logger
= <Logger faust.tables.sets (WARNING)>¶
-
faust.tables.table
¶
Table (key/value changelog stream).
-
class
faust.tables.table.
Table
(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ Table (non-windowed).
-
class
WindowWrapper
(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None] = None, key_index: bool = False, key_index_table: faust.types.tables.TableT = None) → None¶ Windowed table wrapper.
A windowed table does not return concrete values when keys are accessed, instead
WindowSet
is returned so that the values can be further reduced to the wanted time period.-
ValueType
¶ alias of
WindowSet
-
clone
(relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
get_relative_timestamp
¶
-
key_index
= False¶
-
key_index_table
= None¶
-
on_del_key
(key: Any) → None¶ - Return type
None
-
on_recover
(fun: Callable[Awaitable[None]]) → Callable[Awaitable[None]]¶
-
on_set_key
(key: Any, value: Any) → None¶ - Return type
None
-
relative_to
(ts: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_field
(field: faust.types.models.FieldDescriptorT) → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_now
() → faust.types.tables.WindowWrapperT¶ - Return type
-
relative_to_stream
() → faust.types.tables.WindowWrapperT¶ - Return type
-
values
(event: faust.types.events.EventT = None) → ValuesView¶ - Return type
ValuesView
[+VT_co]
-
-
using_window
(window: faust.types.windows.WindowT, *, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
hopping
(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
tumbling
(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
on_key_set
(key: KT, value: VT) → None[source]¶ Handle that value for a key is being set.
- Return type
None
-
logger
= <Logger faust.tables.table (WARNING)>¶
-
class
faust.tables.wrappers
¶
Wrappers for windowed tables.
-
class
faust.tables.wrappers.
WindowedKeysView
(mapping: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None)[source]¶ The object returned by
windowed_table.keys()
.
-
class
faust.tables.wrappers.
WindowedItemsView
(mapping: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None)[source]¶ The object returned by
windowed_table.items()
.
-
class
faust.tables.wrappers.
WindowedValuesView
(mapping: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None)[source]¶ The object returned by
windowed_table.values()
.
-
class
faust.tables.wrappers.
WindowSet
(key: KT, table: faust.types.tables.TableT, wrapper: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None) → None[source]¶ Represents the windows available for table key.
Table[k]
returns WindowSet sincek
can exist in multiple windows, and to retrieve an actual item we need a timestamp.The timestamp of the current event (if this is executing in a stream processor), can be used by accessing
.current()
:Table[k].current()
similarly the most recent value can be accessed using
.now()
:Table[k].now()
from delta of the time of the current event:
Table[k].delta(timedelta(hours=3))
or delta from time of other event:
Table[k].delta(timedelta(hours=3), other_event)
-
apply
(op: Callable[[VT, VT], VT], value: VT, event: faust.types.events.EventT = None) → faust.types.tables.WindowSetT[KT, VT][source]¶ - Return type
WindowSetT
[~KT, ~VT]
-
-
class
faust.tables.wrappers.
WindowWrapper
(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None] = None, key_index: bool = False, key_index_table: faust.types.tables.TableT = None) → None[source]¶ Windowed table wrapper.
A windowed table does not return concrete values when keys are accessed, instead
WindowSet
is returned so that the values can be further reduced to the wanted time period.-
key_index
= False¶
-
key_index_table
= None¶
-
clone
(relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
relative_to
(ts: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
relative_to_field
(field: faust.types.models.FieldDescriptorT) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
values
(event: faust.types.events.EventT = None) → ValuesView[source]¶ - Return type
ValuesView
[+VT_co]
-
Transports¶
faust.transport
¶
faust.transport.base
¶
Base message transport implementation.
The Transport is responsible for:
Holds reference to the app that created it.
Creates new consumers/producers.
To see a reference transport implementation go to:
faust/transport/drivers/aiokafka.py
-
class
faust.transport.base.
Transport
(url: List[yarl.URL], app: faust.types.app.AppT, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ Message transport implementation.
-
class
Consumer
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None¶ Base Consumer.
-
close
() → None¶ - Return type
None
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool¶ Maybe commit the offset for all or specific topics.
-
coroutine
commit_and_end_transactions
(self) → None¶ - Return type
None
-
consumer_stopped_errors
= ()¶
-
flow_active
= True¶
-
coroutine
force_commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool¶ - Return type
-
getmany
(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]]¶ - Return type
AsyncIterator
[Tuple
[TP
,Message
]]
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT]¶ Return list of service dependencies for this service.
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None¶ Call during rebalancing when partitions are being revoked.
- Return type
None
-
coroutine
on_restart
(self) → None¶ Service is being restarted.
- Return type
None
-
coroutine
on_stop
(self) → None¶ Service is being stopped/restarted.
- Return type
None
-
coroutine
on_task_error
(self, exc: BaseException) → None¶ - Return type
None
-
pause_partitions
(tps: Iterable[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
perform_seek
(self) → None¶ - Return type
None
-
resume_flow
() → None¶ - Return type
None
-
resume_partitions
(tps: Iterable[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
seek
(self, partition: faust.types.tuples.TP, offset: int) → None¶ - Return type
None
-
coroutine
seek_to_committed
(self) → Mapping[faust.types.tuples.TP, int]¶
-
stop_flow
() → None¶ - Return type
None
-
track_message
(message: faust.types.tuples.Message) → None¶ - Return type
None
-
coroutine
wait_empty
(self) → None¶ Wait for all messages that started processing to be acked.
- Return type
None
-
-
class
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None¶ Base Producer.
-
coroutine
abort_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
begin_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
commit_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
coroutine
flush
(self) → None¶ - Return type
None
-
logger
= <Logger faust.transport.producer (WARNING)>¶
-
coroutine
maybe_begin_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata]¶ - Return type
-
coroutine
send_and_wait
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → faust.types.tuples.RecordMetadata¶ - Return type
-
coroutine
stop_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
-
class
TransactionManager
(transport: faust.types.transports.TransportT, *, consumer: faust.types.transports.ConsumerT, producer: faust.types.transports.ProducerT, **kwargs) → None¶ -
coroutine
commit
(self, offsets: Mapping[faust.types.tuples.TP, int], start_new_transaction: bool = True) → bool¶ - Return type
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 30.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
coroutine
flush
(self) → None¶ - Return type
None
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
on_rebalance
(self, assigned: Set[faust.types.tuples.TP], revoked: Set[faust.types.tuples.TP], newly_assigned: Set[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata]¶ - Return type
-
coroutine
send_and_wait
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → faust.types.tuples.RecordMetadata¶ - Return type
-
transactional_id_format
= '{tpg.group}-{tpg.partition}'¶
-
coroutine
-
class
Conductor
(app: faust.types.app.AppT, **kwargs) → None¶ Manages the channels that subscribe to topics.
Consumes messages from topic using a single consumer.
Forwards messages to all channels subscribing to a topic.
-
add
(topic: Any) → None¶ Add an element.
- Return type
None
-
clear
() → None¶ This is slow (creates N new iterators!) but effective.
- Return type
None
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool¶ - Return type
-
discard
(topic: Any) → None¶ Remove an element. Do not raise an exception if absent.
- Return type
None
-
logger
= <Logger faust.transport.conductor (WARNING)>¶
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
wait_for_subscriptions
(self) → None¶ - Return type
None
-
class
Fetcher
(app: faust.types.app.AppT, **kwargs) → None¶ Service fetching messages from Kafka.
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
coroutine
on_stop
(self) → None¶ Service is being stopped/restarted.
- Return type
None
-
-
create_consumer
(callback: Callable[faust.types.tuples.Message, Awaitable], **kwargs) → faust.types.transports.ConsumerT[source]¶ - Return type
-
create_transaction_manager
(consumer: faust.types.transports.ConsumerT, producer: faust.types.transports.ProducerT, **kwargs) → faust.types.transports.TransactionManagerT[source]¶ - Return type
-
class
-
class
faust.transport.base.
Conductor
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Manages the channels that subscribe to topics.
Consumes messages from topic using a single consumer.
Forwards messages to all channels subscribing to a topic.
-
logger
= <Logger faust.transport.conductor (WARNING)>¶
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ - Return type
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
class
faust.transport.base.
Consumer
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ Base Consumer.
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
consumer_stopped_errors
= ()¶ Tuple of exception types that may be raised when the underlying consumer driver is stopped.
-
flow_active
= True¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
getmany
(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]][source]¶ - Return type
AsyncIterator
[Tuple
[TP
,Message
]]
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool[source]¶ Maybe commit the offset for all or specific topics.
-
coroutine
force_commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool[source]¶ - Return type
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ Call during rebalancing when partitions are being revoked.
- Return type
None
-
coroutine
seek
(self, partition: faust.types.tuples.TP, offset: int) → None[source]¶ - Return type
None
-
-
class
faust.transport.base.
Fetcher
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Service fetching messages from Kafka.
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
-
class
faust.transport.base.
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ Base Producer.
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None[source]¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
logger
= <Logger faust.transport.producer (WARNING)>¶
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
faust.transport.conductor
¶
The conductor delegates messages from the consumer to the streams.
-
class
faust.transport.conductor.
ConductorCompiler
[source]¶ Compile a function to handle the messages for a topic+partition.
-
class
faust.transport.conductor.
Conductor
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Manages the channels that subscribe to topics.
Consumes messages from topic using a single consumer.
Forwards messages to all channels subscribing to a topic.
-
logger
= <Logger faust.transport.conductor (WARNING)>¶
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ - Return type
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
faust.transport.consumer
¶
Consumer - fetching messages and managing consumer state.
The Consumer is responsible for:
Holds reference to the transport that created it
… and the app via
self.transport.app
.Has a callback that usually points back to
Conductor.on_message
.Receives messages and calls the callback for every message received.
Keeps track of the message and its acked/unacked status.
The Conductor forwards the message to all Streams that subscribes to the topic the message was sent to.
Messages are reference counted, and the Conductor increases the reference count to the number of subscribed streams.
Stream.__aiter__
is set up in a way such that when what is iterating over the stream is finished with the message, a finally: block will decrease the reference count by one.When the reference count for a message hits zero, the stream will call
Consumer.ack(message)
, which will mark that topic + partition + offset combination as “committable”If all the streams share the same key_type/value_type, the conductor will only deserialize the payload once.
Commits the offset at an interval
The Consumer has a background thread that periodically commits the offset.
If the consumer marked an offset as committable this thread will advance the committed offset.
To find the offset that it can safely advance to the commit thread will traverse the _acked mapping of TP to list of acked offsets, by finding a range of consecutive acked offsets (see note in _new_offset).
-
class
faust.transport.consumer.
Fetcher
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Service fetching messages from Kafka.
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
-
class
faust.transport.consumer.
Consumer
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ Base Consumer.
-
logger
= <Logger faust.transport.consumer (WARNING)>¶
-
consumer_stopped_errors
= ()¶ Tuple of exception types that may be raised when the underlying consumer driver is stopped.
-
flow_active
= True¶
-
on_init_dependencies
() → Iterable[mode.types.services.ServiceT][source]¶ Return list of service dependencies for this service.
-
getmany
(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]][source]¶ - Return type
AsyncIterator
[Tuple
[TP
,Message
]]
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool[source]¶ Maybe commit the offset for all or specific topics.
-
coroutine
force_commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool[source]¶ - Return type
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ Call during rebalancing when partitions are being revoked.
- Return type
None
-
coroutine
seek
(self, partition: faust.types.tuples.TP, offset: int) → None[source]¶ - Return type
None
-
faust.transport.producer
¶
Producer.
The Producer is responsible for:
Holds reference to the transport that created it
… and the app via
self.transport.app
.Sending messages.
-
class
faust.transport.producer.
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ Base Producer.
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None[source]¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
logger
= <Logger faust.transport.producer (WARNING)>¶
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
faust.transport.drivers
¶
Transport registry.
faust.transport.drivers.aiokafka
¶
Message transport using aiokafka.
-
class
faust.transport.drivers.aiokafka.
Consumer
(*args, **kwargs) → None[source]¶ Kafka consumer using aiokafka.
-
logger
= <Logger faust.transport.drivers.aiokafka (WARNING)>¶
-
RebalanceListener
¶ alias of
ConsumerRebalanceListener
-
consumer_stopped_errors
= (<class 'aiokafka.errors.ConsumerStoppedError'>,)¶
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 30.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
-
class
faust.transport.drivers.aiokafka.
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ Kafka producer using aiokafka.
-
logger
= <Logger faust.transport.drivers.aiokafka (WARNING)>¶
-
allow_headers
= True¶
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None[source]¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 20.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.transport.drivers.aiokafka.
Transport
(*args, **kwargs) → None[source]¶ Kafka transport using aiokafka.
-
class
Consumer
(*args, **kwargs) → None¶ Kafka consumer using aiokafka.
-
RebalanceListener
¶ alias of
ConsumerRebalanceListener
-
consumer_stopped_errors
= (<class 'aiokafka.errors.ConsumerStoppedError'>,)¶
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 30.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
logger
= <Logger faust.transport.drivers.aiokafka (WARNING)>¶
-
coroutine
on_stop
(self) → None¶ Service is being stopped/restarted.
- Return type
None
-
-
class
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None¶ Kafka producer using aiokafka.
-
coroutine
abort_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
allow_headers
= True¶
-
coroutine
begin_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
commit_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 20.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
coroutine
flush
(self) → None¶ - Return type
None
-
logger
= <Logger faust.transport.drivers.aiokafka (WARNING)>¶
-
coroutine
maybe_begin_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
on_start
(self) → None¶ Service is starting.
- Return type
None
-
coroutine
on_stop
(self) → None¶ Service is being stopped/restarted.
- Return type
None
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata]¶ - Return type
-
coroutine
send_and_wait
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → faust.types.tuples.RecordMetadata¶ - Return type
-
coroutine
stop_transaction
(self, transactional_id: str) → None¶ - Return type
None
-
coroutine
-
default_port
= 9092¶
-
driver_version
= 'aiokafka=1.0.3'¶
-
class
faust.transport.drivers.memory
¶
Experimental: In-memory transport.
-
class
faust.transport.drivers.memory.
RebalanceListener
(*args, **kwargs)[source]¶ In-memory rebalance listener.
-
class
faust.transport.drivers.memory.
Consumer
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ In-memory consumer.
-
class
RebalanceListener
(*args, **kwargs)¶ In-memory rebalance listener.
-
consumer_stopped_errors
= ()¶
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
coroutine
earliest_offsets
(self, *partitions) → MutableMapping[faust.types.tuples.TP, int][source]¶ - Return type
-
getmany
(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]][source]¶ - Return type
AsyncIterator
[Tuple
[TP
,Message
]]
-
coroutine
highwaters
(self, *partitions) → MutableMapping[faust.types.tuples.TP, int][source]¶ - Return type
-
logger
= <Logger faust.transport.drivers.memory (WARNING)>¶
-
class
-
class
faust.transport.drivers.memory.
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ In-memory producer.
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
logger
= <Logger faust.transport.drivers.memory (WARNING)>¶
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
-
class
faust.transport.drivers.memory.
Transport
(*args, **kwargs) → None[source]¶ In-memory transport.
-
class
Consumer
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None¶ In-memory consumer.
-
class
RebalanceListener
(*args, **kwargs)¶ In-memory rebalance listener.
-
consumer_stopped_errors
= ()¶
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
coroutine
earliest_offsets
(self, *partitions) → MutableMapping[faust.types.tuples.TP, int]¶ - Return type
-
getmany
(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]]¶ - Return type
AsyncIterator
[Tuple
[TP
,Message
]]
-
coroutine
highwaters
(self, *partitions) → MutableMapping[faust.types.tuples.TP, int]¶ - Return type
-
logger
= <Logger faust.transport.drivers.memory (WARNING)>¶
-
pause_partitions
(tps: Iterable[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
perform_seek
(self) → None¶ - Return type
None
-
resume_partitions
(partitions: Iterable[faust.types.tuples.TP]) → None¶ - Return type
None
-
coroutine
seek
(self, partition: faust.types.tuples.TP, offset: int) → None¶ - Return type
None
-
coroutine
seek_to_beginning
(self, *partitions) → None¶ - Return type
None
-
coroutine
seek_to_latest
(self, *partitions) → None¶ - Return type
None
-
coroutine
subscribe
(self, topics: Iterable[str]) → None¶ - Return type
None
-
class
-
class
Producer
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None¶ In-memory producer.
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None¶ - Return type
None
-
logger
= <Logger faust.transport.drivers.memory (WARNING)>¶
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata]¶ - Return type
-
coroutine
send_and_wait
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → faust.types.tuples.RecordMetadata¶ - Return type
-
coroutine
-
default_port
= 9092¶
-
driver_version
= 'memory-1.5.5'¶
-
class
faust.transport.utils
¶
Transport utils - scheduling.
-
faust.transport.utils.
TopicIndexMap
¶ alias of
typing.MutableMapping
-
class
faust.transport.utils.
DefaultSchedulingStrategy
[source]¶ Consumer record scheduler.
Delivers records in round robin between both topics and partitions.
-
classmethod
map_from_records
(records: Mapping[faust.types.tuples.TP, List]) → MutableMapping[str, faust.transport.utils.TopicBuffer][source]¶ - Return type
-
classmethod
Assignor¶
faust.assignor.client_assignment
¶
Client Assignment.
-
class
faust.assignor.client_assignment.
CopartitionedAssignment
(actives: Set[int] = None, standbys: Set[int] = None, topics: Set[str] = None) → None[source]¶ Copartitioned Assignment.
-
class
faust.assignor.client_assignment.
ClientAssignment
(actives, standbys, *, __strict__=True, __faust=None, **kwargs) → None[source]¶ Client Assignment data model.
-
actives
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
standbys
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
kafka_protocol_assignment
(table_manager: faust.types.tables.TableManagerT) → Sequence[Tuple[str, List[int]]][source]¶
-
add_copartitioned_assignment
(assignment: faust.assignor.client_assignment.CopartitionedAssignment) → None[source]¶ - Return type
None
-
copartitioned_assignment
(topics: Set[str]) → faust.assignor.client_assignment.CopartitionedAssignment[source]¶ - Return type
-
asdict
()¶
-
-
class
faust.assignor.client_assignment.
ClientMetadata
(assignment, url, changelog_distribution, topic_groups=None, *, __strict__=True, __faust=None, **kwargs) → None[source]¶ Client Metadata data model.
-
assignment
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
url
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
asdict
()¶
-
changelog_distribution
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
topic_groups
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
faust.assignor.cluster_assignment
¶
Cluster assignment.
-
faust.assignor.cluster_assignment.
CopartMapping
¶ alias of
typing.MutableMapping
-
class
faust.assignor.cluster_assignment.
ClusterAssignment
(subscriptions=None, assignments=None, *, __strict__=True, __faust=None, **kwargs) → None[source]¶ Cluster assignment state.
-
subscriptions
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
assignments
¶ Describes a field.
Used for every field in Record so that they can be used in join’s /group_by etc.
Examples
>>> class Withdrawal(Record): ... account_id: str ... amount: float = 0.0
>>> Withdrawal.account_id <FieldDescriptor: Withdrawal.account_id: str> >>> Withdrawal.amount <FieldDescriptor: Withdrawal.amount: float = 0.0>
-
add_client
(client: str, subscription: List[str], metadata: faust.assignor.client_assignment.ClientMetadata) → None[source]¶ - Return type
None
-
copartitioned_assignments
(copartitioned_topics: Set[str]) → MutableMapping[str, faust.assignor.client_assignment.CopartitionedAssignment][source]¶ - Return type
-
asdict
()¶
-
faust.assignor.copartitioned_assignor
¶
Copartitioned Assignor.
-
class
faust.assignor.copartitioned_assignor.
CopartitionedAssignor
(topics: Iterable[str], cluster_asgn: MutableMapping[str, faust.assignor.client_assignment.CopartitionedAssignment], num_partitions: int, replicas: int, capacity: int = None) → None[source]¶ Copartitioned Assignor.
All copartitioned topics must have the same number of partitions
The assignment is sticky which uses the following heuristics:
Maintain existing assignments as long as within capacity for each client
Assign actives to standbys when possible (within capacity)
Assign in order to fill capacity of the clients
We optimize for not over utilizing resources instead of under-utilizing resources. This results in a balanced assignment when capacity is the default value which is
ceil(num partitions / num clients)
Notes
Currently we raise an exception if number of clients is not enough for the desired replication.
faust.assignor.leader_assignor
¶
Leader assignor.
faust.assignor.partition_assignor
¶
Partition assignor.
-
faust.assignor.partition_assignor.
MemberAssignmentMapping
¶ alias of
typing.MutableMapping
-
faust.assignor.partition_assignor.
MemberMetadataMapping
¶ alias of
typing.MutableMapping
-
faust.assignor.partition_assignor.
MemberSubscriptionMapping
¶ alias of
typing.MutableMapping
-
faust.assignor.partition_assignor.
ClientMetadataMapping
¶ alias of
typing.MutableMapping
-
faust.assignor.partition_assignor.
ClientAssignmentMapping
¶ alias of
typing.MutableMapping
-
faust.assignor.partition_assignor.
CopartitionedGroups
¶ alias of
typing.MutableMapping
-
class
faust.assignor.partition_assignor.
PartitionAssignor
(app: faust.types.app.AppT, replicas: int = 0) → None[source]¶ PartitionAssignor handles internal topic creation.
Further, this assignor needs to be sticky and potentially redundant
Notes
Interface copied from
kafka.coordinator.assignors.abstract
.-
changelog_distribution
¶ - Return type
-
on_assignment
(assignment: kafka.coordinator.protocol.ConsumerProtocolMemberMetadata) → None[source]¶ Callback that runs on each assignment.
This method can be used to update internal state, if any, of the partition assignor.
- Parameters
assignment (MemberAssignment) – the member’s assignment
- Return type
None
-
metadata
(topics: Set[str]) → kafka.coordinator.protocol.ConsumerProtocolMemberMetadata[source]¶ Generate ProtocolMetadata to be submitted via JoinGroupRequest.
- Parameters
topics (set) – a member’s subscribed topics
- Return type
ConsumerProtocolMemberMetadata
- Returns
MemberMetadata struct
-
assign
(cluster: kafka.cluster.ClusterMetadata, member_metadata: MutableMapping[str, kafka.coordinator.protocol.ConsumerProtocolMemberMetadata]) → MutableMapping[str, kafka.coordinator.protocol.ConsumerProtocolMemberAssignment][source]¶ Perform group assignment given cluster metadata and member subscriptions
- Parameters
cluster (ClusterMetadata) – metadata for use in assignment
(dict of {member_id (members) – MemberMetadata}): decoded metadata for each member in the group.
- Return type
MutableMapping
[str
,ConsumerProtocolMemberAssignment
]- Returns
{member_id: MemberAssignment}
- Return type
-
table_metadata
(topic: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]¶ - Return type
-
Types¶
faust.types.agents
¶
-
faust.types.agents.
AgentErrorHandler
¶ alias of
typing.Callable
-
faust.types.agents.
AgentFun
¶ alias of
typing.Callable
-
faust.types.agents.
SinkT
= typing.Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]¶ Agent, Channel or callable/async callable taking value as argument.
- Type
A sink can be
-
class
faust.types.agents.
ActorT
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ -
index
= None¶ If multiple instance are started for concurrency, this is its index.
-
-
class
faust.types.agents.
AsyncIterableActorT
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ Used for agent function that yields.
-
class
faust.types.agents.
AwaitableActorT
(agent: faust.types.agents.AgentT, stream: faust.types.streams.StreamT, it: _T, active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → None[source]¶ Used for agent function that do not yield.
-
faust.types.agents.
ActorRefT
¶ alias of
faust.types.agents.ActorT
-
class
faust.types.agents.
AgentT
(fun: Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], *, name: str = None, app: faust.types.agents._AppT = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, on_error: Callable[[AgentT, BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, isolated_partitions: bool = False, **kwargs) → None[source]¶ -
test_context
(channel: faust.types.channels.ChannelT = None, supervisor_strategy: mode.types.supervisors.SupervisorStrategyT = None, **kwargs) → faust.types.agents.AgentTestWrapperT[source]¶ - Return type
-
add_sink
(sink: Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]) → None[source]¶ - Return type
None
-
clone
(*, cls: Type[AgentT] = None, **kwargs) → faust.types.agents.AgentT[source]¶ - Return type
AgentT
[]
-
channel_iterator
¶ - Return type
AsyncIterator
[+T_co]
-
coroutine
ask
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]¶ - Return type
-
coroutine
cast
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → None[source]¶ - Return type
None
-
coroutine
join
(self, values: Union[AsyncIterable[Union[bytes, faust.types.core._ModelT, Any]], Iterable[Union[bytes, faust.types.core._ModelT, Any]]], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvjoin
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → List[Any][source]¶
-
coroutine
kvmap
(self, items: Union[AsyncIterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]], Iterable[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], Union[bytes, faust.types.core._ModelT, Any]]]], reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]¶
-
coroutine
map
(self, values: Union[AsyncIterable, Iterable], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None) → AsyncIterator[source]¶
-
coroutine
on_partitions_assigned
(self, assigned: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.types.agents.
AgentManagerT
(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶
-
class
faust.types.agents.
AgentTestWrapperT
(*args, original_channel: faust.types.channels.ChannelT = None, **kwargs) → None[source]¶ -
sent_offset
= 0¶
-
processed_offset
= 0¶
-
coroutine
put
(self, value: Union[bytes, faust.types.core._ModelT, Any] = None, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, *, reply_to: Union[AgentT, faust.types.channels.ChannelT, str] = None, correlation_id: str = None, wait: bool = True) → faust.types.events.EventT[source]¶ - Return type
EventT
[]
-
to_message
(key: Union[bytes, faust.types.core._ModelT, Any, None], value: Union[bytes, faust.types.core._ModelT, Any], *, partition: int = 0, offset: int = 0, timestamp: float = None, timestamp_type: int = 0, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None) → faust.types.tuples.Message[source]¶ - Return type
-
faust.types.app
¶
-
class
faust.types.app.
AppT
(id: str, *, monitor: faust.types.app._Monitor, config_source: Any = None, **options) → None[source]¶ Abstract type for the Faust application.
See also
-
finalized
= False¶ Set to true when the app is finalized (can read configuration).
-
configured
= False¶ Set to true when the app has read configuration.
-
rebalancing
= False¶ Set to true if the worker is currently rebalancing.
-
rebalancing_count
= 0¶
-
unassigned
= False¶
-
in_worker
= False¶
-
on_configured
(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.on_configured>¶
-
on_before_configured
(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.on_before_configured>¶
-
on_after_configured
(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.on_after_configured>¶
-
on_partitions_assigned
(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.on_partitions_assigned>¶
-
on_partitions_revoked
(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.on_partitions_revoked>¶
-
on_rebalance_complete
(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.on_rebalance_complete>¶
-
on_before_shutdown
(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.on_before_shutdown>¶
-
on_worker_init
(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.on_worker_init>¶
-
config_from_object
(obj: Any, *, silent: bool = False, force: bool = False) → None[source]¶ - Return type
None
-
discover
(*extra_modules, categories: Iterable[str] = ('a', 'b', 'c'), ignore: Iterable[Any] = ('foo', 'bar')) → None[source]¶ - Return type
None
-
topic
(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: faust.types.app._ModelArg = None, value_type: faust.types.app._ModelArg = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, maxsize: int = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
channel
(*, key_type: faust.types.app._ModelArg = None, value_type: faust.types.app._ModelArg = None, maxsize: int = None, loop: asyncio.events.AbstractEventLoop = None) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
agent
(channel: Union[str, faust.types.channels.ChannelT] = None, *, name: str = None, concurrency: int = 1, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, sink: Iterable[Union[AgentT, faust.types.channels.ChannelT, Callable[Any, Optional[Awaitable]]]] = None, isolated_partitions: bool = False, use_reply_headers: bool = False, **kwargs) → Callable[Callable[faust.types.streams.StreamT, Union[Coroutine[[Any, Any], None], Awaitable[None], AsyncIterable]], faust.types.agents.AgentT][source]¶
-
task
(fun: Union[Callable[AppT, Awaitable], Callable[Awaitable]], *, on_leader: bool = False, traced: bool = True) → Callable[source]¶
-
timer
(interval: Union[datetime.timedelta, float, str], on_leader: bool = False, traced: bool = True, name: str = None, max_drift_correction: float = 0.1) → Callable[source]¶ - Return type
-
crontab
(cron_format: str, *, timezone: datetime.tzinfo = None, on_leader: bool = False, traced: bool = True) → Callable[source]¶ - Return type
-
stream
(channel: AsyncIterable, beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
Table
(name: str, *, default: Callable[Any] = None, window: faust.types.windows.WindowT = None, partitions: int = None, help: str = None, **kwargs) → faust.types.tables.TableT[source]¶ - Return type
TableT
[~KT, ~VT]
-
page
(path: str, *, base: Type[faust.types.web.View] = <class 'faust.types.web.View'>, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, name: str = None) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Type[faust.types.web.View]][source]¶
-
table_route
(table: faust.types.tables.CollectionT, shard_param: str = None, *, query_param: str = None, match_info: str = None) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
command
(*options, base: Type[faust.types.app._AppCommand] = None, **kwargs) → Callable[Callable, Type[faust.types.app._AppCommand]][source]¶
-
FlowControlQueue
(maxsize: int = None, *, clear_on_resume: bool = False, loop: asyncio.events.AbstractEventLoop = None) → mode.utils.queues.ThrowableQueue[source]¶ - Return type
-
conf
¶ - Return type
_Settings
-
transport
¶ - Return type
-
cache
¶ - Return type
-
monitor
¶ - Return type
_Monitor
-
http_client
¶ - Return type
ClientSession
-
assignor
¶ - Return type
-
coroutine
send
(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
faust.types.assignor
¶
-
faust.types.assignor.
TopicToPartitionMap
¶ alias of
typing.MutableMapping
-
faust.types.assignor.
HostToPartitionMap
¶ alias of
typing.MutableMapping
-
class
faust.types.assignor.
PartitionAssignorT
(app: faust.types.assignor._AppT, replicas: int = 0) → None[source]¶ -
-
table_metadata
(topic: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]¶ - Return type
-
faust.types.auth
¶
-
class
faust.types.auth.
AuthProtocol
[source]¶ An enumeration.
-
SSL
= 'SSL'¶
-
PLAINTEXT
= 'PLAINTEXT'¶
-
SASL_PLAINTEXT
= 'SASL_PLAINTEXT'¶
-
SASL_SSL
= 'SASL_SSL'¶
-
faust.types.channels
¶
-
class
faust.types.channels.
ChannelT
(app: faust.types.channels._AppT, *, key_type: faust.types.channels._ModelArg = None, value_type: faust.types.channels._ModelArg = None, is_iterator: bool = False, queue: mode.utils.queues.ThrowableQueue = None, maxsize: int = None, root: Optional[faust.types.channels.ChannelT] = None, active_partitions: Set[faust.types.tuples.TP] = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
clone
(*, is_iterator: bool = None, **kwargs) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
clone_using_queue
(queue: asyncio.queues.Queue) → faust.types.channels.ChannelT[source]¶ - Return type
ChannelT
[]
-
as_future_message
(key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None) → faust.types.tuples.FutureMessage[source]¶ - Return type
-
prepare_key
(key: Union[bytes, faust.types.core._ModelT, Any, None], key_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
prepare_value
(value: Union[bytes, faust.types.core._ModelT, Any], value_serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ - Return type
-
queue
¶ - Return type
-
coroutine
decode
(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.channels._EventT[source]¶ - Return type
_EventT
-
coroutine
get
(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]¶ - Return type
-
coroutine
on_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_key_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
on_value_decode_error
(self, exc: Exception, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
coroutine
publish_message
(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, *, key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
faust.types.codecs
¶
faust.types.core
¶
-
faust.types.core.
K
= typing.Union[bytes, faust.types.core._ModelT, typing.Any, NoneType]¶ Shorthand for the type of a key
-
faust.types.core.
V
= typing.Union[bytes, faust.types.core._ModelT, typing.Any]¶ Shorthand for the type of a value
faust.types.enums
¶
faust.types.events
¶
-
class
faust.types.events.
EventT
(app: faust.types.events._AppT, key: Union[bytes, faust.types.core._ModelT, Any, None], value: Union[bytes, faust.types.core._ModelT, Any], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], message: faust.types.tuples.Message) → None[source]¶ -
app
¶
-
key
¶
-
value
¶
-
headers
¶
-
message
¶
-
acked
¶
-
coroutine
forward
(self, channel: Union[str, faust.types.events._ChannelT], key: Any = None, value: Any = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
coroutine
send
(self, channel: Union[str, faust.types.events._ChannelT], key: Union[bytes, faust.types.core._ModelT, Any, None] = None, value: Union[bytes, faust.types.core._ModelT, Any] = None, partition: int = None, timestamp: float = None, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[None, Awaitable[None]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
faust.types.fixups
¶
faust.types.joins
¶
faust.types.models
¶
-
faust.types.models.
CoercionHandler
¶ alias of
typing.Callable
-
class
faust.types.models.
TypeCoerce
(*args, **kwargs)[source]¶ -
target
¶ Alias for field number 0
-
handler
¶ Alias for field number 1
-
-
class
faust.types.models.
ModelOptions
(*args, **kwargs)[source]¶ -
serializer
= None¶
-
include_metadata
= True¶
-
allow_blessed_key
= False¶
-
isodates
= False¶
-
decimals
= False¶
-
coercions
= None¶
-
fields
= None¶ Flattened view of __annotations__ in MRO order.
- Type
Index
-
fieldset
= None¶ Set of required field names, for fast argument checking.
- Type
Index
-
fieldpos
= None¶ Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.
- Type
Index
-
optionalset
= None¶ Set of optional field names, for fast argument checking.
- Type
Index
-
models
= None¶ Mapping of fields that are ModelT
- Type
Index
-
modelattrs
= None¶
-
field_coerce
= None¶ Mapping of fields that need to be coerced. Key is the name of the field, value is the coercion handler function.
- Type
Index
-
defaults
= None¶ Mapping of field names to default value.
-
initfield
= None¶ Mapping of init field conversion callbacks.
-
-
class
faust.types.models.
ModelT
(*args, **kwargs) → None[source]¶ -
classmethod
from_data
(data: Any, *, preferred_type: Type[ModelT] = None) → faust.types.models.ModelT[source]¶ - Return type
-
classmethod
loads
(s: bytes, *, default_serializer: Union[faust.types.codecs.CodecT, str, None] = None, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → faust.types.models.ModelT[source]¶ - Return type
-
classmethod
faust.types.router
¶
Types for module faust.router
.
-
class
faust.types.router.
RouterT
(app: faust.types.router._AppT) → None[source]¶ Router type class.
-
key_store
(table_name: str, key: Union[bytes, faust.types.core._ModelT, Any, None]) → yarl.URL[source]¶ - Return type
-
table_metadata
(table_name: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]¶ - Return type
-
faust.types.sensors
¶
-
class
faust.types.sensors.
SensorInterfaceT
[source]¶ -
on_message_in
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_stream_event_in
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → Optional[Dict][source]¶
-
on_stream_event_out
(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT, state: Dict = None) → None[source]¶ - Return type
None
-
on_message_out
(tp: faust.types.tuples.TP, offset: int, message: faust.types.tuples.Message) → None[source]¶ - Return type
None
-
on_table_set
(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]¶ - Return type
None
-
on_commit_completed
(consumer: faust.types.transports.ConsumerT, state: Any) → None[source]¶ - Return type
None
-
on_send_initiated
(producer: faust.types.transports.ProducerT, topic: str, message: faust.types.tuples.PendingMessage, keysize: int, valsize: int) → Any[source]¶ - Return type
-
on_send_completed
(producer: faust.types.transports.ProducerT, state: Any, metadata: faust.types.tuples.RecordMetadata) → None[source]¶ - Return type
None
-
on_send_error
(producer: faust.types.transports.ProducerT, exc: BaseException, state: Any) → None[source]¶ - Return type
None
-
on_assignment_start
(assignor: faust.types.assignor.PartitionAssignorT) → Dict[source]¶ - Return type
Dict
[~KT, ~VT]
-
on_assignment_error
(assignor: faust.types.assignor.PartitionAssignorT, state: Dict, exc: BaseException) → None[source]¶ - Return type
None
-
faust.types.serializers
¶
-
class
faust.types.serializers.
RegistryT
(key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = 'json') → None[source]¶ -
loads_key
(typ: Optional[faust.types.serializers._ModelArg], key: Optional[bytes], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → Union[bytes, faust.types.core._ModelT, Any, None][source]¶
-
loads_value
(typ: Optional[faust.types.serializers._ModelArg], value: Optional[bytes], *, serializer: Union[faust.types.codecs.CodecT, str, None] = None) → Any[source]¶ - Return type
-
faust.types.settings
¶
-
class
faust.types.settings.
Settings
(id: str, *, version: int = None, broker: Union[str, yarl.URL, List[yarl.URL]] = None, broker_client_id: str = None, broker_request_timeout: Union[datetime.timedelta, float, str] = None, broker_credentials: Union[faust.types.auth.CredentialsT, ssl.SSLContext] = None, broker_commit_every: int = None, broker_commit_interval: Union[datetime.timedelta, float, str] = None, broker_commit_livelock_soft_timeout: Union[datetime.timedelta, float, str] = None, broker_session_timeout: Union[datetime.timedelta, float, str] = None, broker_heartbeat_interval: Union[datetime.timedelta, float, str] = None, broker_check_crcs: bool = None, broker_max_poll_records: int = None, agent_supervisor: Union[_T, str] = None, store: Union[str, yarl.URL] = None, cache: Union[str, yarl.URL] = None, web: Union[str, yarl.URL] = None, web_enabled: bool = True, processing_guarantee: Union[str, faust.types.enums.ProcessingGuarantee] = None, timezone: datetime.tzinfo = None, autodiscover: Union[bool, Iterable[str], Callable[Iterable[str]]] = None, origin: str = None, canonical_url: Union[str, yarl.URL] = None, datadir: Union[pathlib.Path, str] = None, tabledir: Union[pathlib.Path, str] = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, logging_config: Dict = None, loghandlers: List[logging.Handler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = None, topic_allow_declare: bool = None, id_format: str = None, reply_to: str = None, reply_to_prefix: str = None, reply_create_topic: bool = None, reply_expires: Union[datetime.timedelta, float, str] = None, ssl_context: ssl.SSLContext = None, stream_buffer_maxsize: int = None, stream_wait_empty: bool = None, stream_ack_cancelled_tasks: bool = None, stream_ack_exceptions: bool = None, stream_publish_on_commit: bool = None, stream_recovery_delay: Union[datetime.timedelta, float, str] = None, producer_linger_ms: int = None, producer_max_batch_size: int = None, producer_acks: int = None, producer_max_request_size: int = None, producer_compression_type: str = None, producer_partitioner: Union[_T, str] = None, producer_request_timeout: Union[datetime.timedelta, float, str] = None, producer_api_version: str = None, consumer_max_fetch_size: int = None, consumer_auto_offset_reset: str = None, web_bind: str = None, web_port: int = None, web_host: str = None, web_transport: Union[str, yarl.URL] = None, web_in_thread: bool = None, web_cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[_T, str] = None, ConsumerScheduler: Union[_T, str] = None, Stream: Union[_T, str] = None, Table: Union[_T, str] = None, SetTable: Union[_T, str] = None, TableManager: Union[_T, str] = None, Serializers: Union[_T, str] = None, Worker: Union[_T, str] = None, PartitionAssignor: Union[_T, str] = None, LeaderAssignor: Union[_T, str] = None, Router: Union[_T, str] = None, Topic: Union[_T, str] = None, HttpClient: Union[_T, str] = None, Monitor: Union[_T, str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None[source]¶ -
-
id_format
= '{id}-v{self.version}'¶
-
ssl_context
= None¶
-
autodiscover
= False¶
-
broker_client_id
= 'faust-1.5.5'¶
-
timezone
= datetime.timezone.utc¶
-
broker_commit_every
= 10000¶
-
broker_check_crcs
= True¶
-
key_serializer
= 'raw'¶
-
value_serializer
= 'json'¶
-
table_standby_replicas
= 1¶
-
topic_replication_factor
= 1¶
-
topic_partitions
= 8¶
-
topic_allow_declare
= True¶
-
reply_create_topic
= False¶
-
logging_config
= None¶
-
stream_buffer_maxsize
= 4096¶
-
stream_wait_empty
= True¶
-
stream_ack_cancelled_tasks
= True¶
-
stream_ack_exceptions
= True¶
-
stream_publish_on_commit
= False¶
-
producer_linger_ms
= 0¶
-
producer_max_batch_size
= 16384¶
-
producer_acks
= -1¶
-
producer_max_request_size
= 1000000¶
-
producer_compression_type
= None¶
-
producer_api_version
= 'auto'¶
-
consumer_max_fetch_size
= 4194304¶
-
consumer_auto_offset_reset
= 'earliest'¶
-
web_bind
= '0.0.0.0'¶
-
web_port
= 6066¶
-
web_host
= 'build-8929922-project-230058-faust'¶
-
web_in_thread
= False¶
-
web_cors_options
= None¶
-
worker_redirect_stdouts
= True¶
-
worker_redirect_stdouts_level
= 'WARN'¶
-
reply_to_prefix
= 'f-reply-'¶
-
processing_guarantee
¶ - Return type
-
broker_credentials
¶ - Return type
-
producer_partitioner
¶
-
agent_supervisor
¶ - Return type
-
TableManager
¶ - Return type
-
PartitionAssignor
¶ - Return type
-
LeaderAssignor
¶ - Return type
-
faust.types.stores
¶
-
class
faust.types.stores.
StoreT
(url: Union[str, yarl.URL], app: faust.types.stores._AppT, table: faust.types.stores._CollectionT, *, table_name: str = '', key_type: faust.types.stores._ModelArg = None, value_type: faust.types.stores._ModelArg = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = '', value_serializer: Union[faust.types.codecs.CodecT, str, None] = '', **kwargs) → None[source]¶ -
-
apply_changelog_batch
(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, KT], to_value: Callable[Any, VT]) → None[source]¶ - Return type
None
-
faust.types.streams
¶
-
faust.types.streams.
Processor
¶ alias of
typing.Callable
-
faust.types.streams.
GroupByKeyArg
= typing.Union[faust.types.models.FieldDescriptorT, typing.Callable[[~T], typing.Union[bytes, faust.types.core._ModelT, typing.Any, NoneType]]]¶ Type of the key argument to Stream.group_by()
-
class
faust.types.streams.
StreamT
(channel: AsyncIterator[T_co] = None, *, app: faust.types.streams._AppT = None, processors: Iterable[Callable[T]] = None, combined: List[faust.types.streams.JoinableT] = None, on_start: Callable = None, join_strategy: faust.types.streams._JoinT = None, beacon: mode.utils.types.trees.NodeT = None, concurrency_index: int = None, prev: Optional[faust.types.streams.StreamT] = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, prefix: str = '', loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
outbox
= None¶
-
join_strategy
= None¶
-
task_owner
= None¶
-
current_event
= None¶
-
active_partitions
= None¶
-
concurrency_index
= None¶
-
enable_acks
= True¶
-
prefix
= ''¶
-
enumerate
(start: int = 0) → AsyncIterable[Tuple[int, T_co]][source]¶ - Return type
AsyncIterable
[Tuple
[int
, +T_co]]
-
through
(channel: Union[str, faust.types.channels.ChannelT]) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
group_by
(key: Union[faust.types.models.FieldDescriptorT, Callable[T, Union[bytes, faust.types.core._ModelT, Any, None]]], *, name: str = None, topic: faust.types.topics.TopicT = None) → faust.types.streams.StreamT[source]¶ - Return type
StreamT
[+T_co]
-
derive_topic
(name: str, *, key_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, value_type: Union[Type[faust.types.models.ModelT], Type[bytes], Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
coroutine
items
(self) → AsyncIterator[Tuple[Union[bytes, faust.types.core._ModelT, Any, None], T_co]][source]¶
-
faust.types.tables
¶
-
faust.types.tables.
RecoverCallback
¶ alias of
typing.Callable
-
faust.types.tables.
ChangelogEventCallback
¶ alias of
typing.Callable
-
faust.types.tables.
CollectionTps
¶ alias of
typing.MutableMapping
-
class
faust.types.tables.
CollectionT
(app: faust.types.tables._AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: faust.types.tables._ModelArg = None, value_type: faust.types.tables._ModelArg = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ -
-
coroutine
on_changelog_event
(self, event: faust.types.events.EventT) → None[source]¶ - Return type
None
-
coroutine
-
class
faust.types.tables.
TableT
(app: faust.types.tables._AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: faust.types.tables._ModelArg = None, value_type: faust.types.tables._ModelArg = None, partitions: int = None, window: faust.types.windows.WindowT = None, changelog_topic: faust.types.topics.TopicT = None, help: str = None, on_recover: Callable[Awaitable[None]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[None]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]¶ -
using_window
(window: faust.types.windows.WindowT, *, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
hopping
(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None, key_index: bool = False) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
-
class
faust.types.tables.
TableManagerT
(app: faust.types.tables._AppT, **kwargs) → None[source]¶
-
class
faust.types.tables.
WindowSetT
(key: KT, table: faust.types.tables.TableT, wrapper: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None) → None[source]¶ -
apply
(op: Callable[[VT, VT], VT], value: VT, event: faust.types.events.EventT = None) → faust.types.tables.WindowSetT[source]¶ - Return type
WindowSetT
[~KT, ~VT]
-
-
class
faust.types.tables.
WindowedItemsViewT
(mapping: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None)[source]¶
-
class
faust.types.tables.
WindowedValuesViewT
(mapping: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None)[source]¶
-
class
faust.types.tables.
WindowWrapperT
(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None] = None) → None[source]¶ -
-
clone
(relative_to: Union[faust.types.tables._FieldDescriptorT, Callable[Optional[faust.types.events.EventT], Union[float, datetime.datetime]], datetime.datetime, float, None]) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
relative_to_field
(field: faust.types.tables._FieldDescriptorT) → faust.types.tables.WindowWrapperT[source]¶ - Return type
-
faust.types.topics
¶
-
class
faust.types.topics.
TopicT
(app: faust.types.topics._AppT, *, topics: Sequence[str] = None, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: faust.types.topics._ModelArg = None, value_type: faust.types.topics._ModelArg = None, is_iterator: bool = False, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, replicas: int = None, acks: bool = True, internal: bool = False, config: Mapping[str, Any] = None, queue: mode.utils.queues.ThrowableQueue = None, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, maxsize: int = None, root: faust.types.channels.ChannelT = None, active_partitions: Set[faust.types.tuples.TP] = None, allow_empty: bool = False, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
topics
= None¶ Iterable/Sequence of topic names to subscribe to.
-
retention
= None¶ expiry time in seconds for messages in the topic.
- Type
Topic retention setting
-
compacting
= None¶ Flag that when enabled means the topic can be “compacted”: if the topic is a log of key/value pairs, the broker can delete old values for the same key.
-
replicas
= None¶ Number of replicas for topic.
-
config
= None¶ Additional configuration as a mapping.
-
acks
= None¶ Enable acks for this topic.
-
internal
= None¶ it’s owned by us and we are allowed to create or delete the topic as necessary.
- Type
Mark topic as internal
-
pattern
¶ or instead of
topics
, a regular expression used to match topics we want to subscribe to. :rtype:Optional
[Pattern
[AnyStr
]]
-
derive_topic
(*, topics: Sequence[str] = None, key_type: faust.types.topics._ModelArg = None, value_type: faust.types.topics._ModelArg = None, partitions: int = None, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, internal: bool = False, config: Mapping[str, Any] = None, prefix: str = '', suffix: str = '', **kwargs) → faust.types.topics.TopicT[source]¶ - Return type
TopicT
[]
-
faust.types.transports
¶
-
faust.types.transports.
ConsumerCallback
¶ alias of
typing.Callable
-
faust.types.transports.
TPorTopicSet
¶ alias of
typing.AbstractSet
-
faust.types.transports.
PartitionsRevokedCallback
¶ alias of
typing.Callable
-
faust.types.transports.
PartitionsAssignedCallback
¶ alias of
typing.Callable
-
faust.types.transports.
PartitionerT
¶ alias of
typing.Callable
-
class
faust.types.transports.
ProducerT
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ -
transport
= None¶ The transport that created this Producer.
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None[source]¶ - Return type
None
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
coroutine
send
(self, topic: str, key: Optional[bytes], value: Optional[bytes], partition: Optional[int], timestamp: Optional[float], headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], *, transactional_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]¶ - Return type
-
-
class
faust.types.transports.
TransactionManagerT
(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, *, consumer: faust.types.transports.ConsumerT, producer: faust.types.transports.ProducerT, **kwargs) → None[source]¶ -
-
coroutine
commit
(self, offsets: Mapping[faust.types.tuples.TP, int], start_new_transaction: bool = True) → bool[source]¶ - Return type
-
coroutine
commit_transactions
(self, tid_to_offset_map: Mapping[str, Mapping[faust.types.tuples.TP, int]], group_id: str, start_new_transaction: bool = True) → None[source]¶ - Return type
None
-
coroutine
on_partitions_revoked
(self, revoked: Set[faust.types.tuples.TP]) → None[source]¶ - Return type
None
-
coroutine
-
class
faust.types.transports.
ConsumerT
(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[None]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[None]], *, commit_interval: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]¶ -
transport
= None¶ The transport that created this Consumer.
-
commit_interval
= None¶ How often we commit topic offsets. See
broker_commit_interval
.
-
randomly_assigned_topics
= None¶ Set of topic names that are considered “randomly assigned”. This means we don’t crash if it’s not part of our assignment. Used by e.g. the leader assignor service.
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None, start_new_transaction: bool = True) → bool[source]¶ - Return type
-
coroutine
create_topic
(self, topic: str, partitions: int, replication: int, *, config: Mapping[str, Any] = None, timeout: Union[datetime.timedelta, float, str] = 1000.0, retention: Union[datetime.timedelta, float, str] = None, compacting: bool = None, deleting: bool = None, ensure_created: bool = False) → None[source]¶ - Return type
None
-
coroutine
getmany
(self, timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]][source]¶
-
coroutine
seek
(self, partition: faust.types.tuples.TP, offset: int) → None[source]¶ - Return type
None
-
-
class
faust.types.transports.
ConductorT
(app: faust.types.transports._AppT, **kwargs) → None[source]¶ -
-
coroutine
commit
(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]¶ - Return type
-
coroutine
-
class
faust.types.transports.
TransportT
(url: List[yarl.URL], app: faust.types.transports._AppT, loop: asyncio.events.AbstractEventLoop = None) → None[source]¶ -
Consumer
= None¶ The Consumer class used for this type of transport.
-
Producer
= None¶ The Producer class used for this type of transport.
-
TransactionManager
= None¶ The TransactionManager class used for managing multiple transactions.
-
Conductor
= None¶ The Conductor class used to delegate messages from Consumer to streams.
-
Fetcher
= None¶ The Fetcher service used for this type of transport.
-
url
= None¶ //localhost).
- Type
The URL to use for this transport (e.g. kafka
-
driver_version
= None¶ String identifying the underlying driver used for this transport. E.g. for aiokafka this could be
aiokafka 0.4.1
.
-
create_consumer
(callback: Callable[faust.types.tuples.Message, Awaitable], **kwargs) → faust.types.transports.ConsumerT[source]¶ - Return type
-
create_transaction_manager
(consumer: faust.types.transports.ConsumerT, producer: faust.types.transports.ProducerT, **kwargs) → faust.types.transports.TransactionManagerT[source]¶ - Return type
-
faust.types.tuples
¶
-
class
faust.types.tuples.
TP
(*args, **kwargs)[source]¶ -
topic
¶ Alias for field number 0
-
partition
¶ Alias for field number 1
-
-
class
faust.types.tuples.
RecordMetadata
(*args, **kwargs)[source]¶ -
topic
¶ Alias for field number 0
-
partition
¶ Alias for field number 1
-
topic_partition
¶ Alias for field number 2
-
offset
¶ Alias for field number 3
-
timestamp
¶ Alias for field number 4
-
timestamp_type
¶ Alias for field number 5
-
-
class
faust.types.tuples.
PendingMessage
(*args, **kwargs)[source]¶ -
channel
¶ Alias for field number 0
-
key
¶ Alias for field number 1
-
value
¶ Alias for field number 2
-
partition
¶ Alias for field number 3
-
timestamp
¶ Alias for field number 4
-
headers
¶ Alias for field number 5
-
key_serializer
¶ Alias for field number 6
-
value_serializer
¶ Alias for field number 7
-
callback
¶ Alias for field number 8
-
topic
¶ Alias for field number 9
-
offset
¶ Alias for field number 10
-
-
class
faust.types.tuples.
Message
(topic: str, partition: int, offset: int, timestamp: float, timestamp_type: int, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], key: Optional[bytes], value: Optional[bytes], checksum: Optional[bytes], serialized_key_size: int = None, serialized_value_size: int = None, tp: faust.types.tuples.TP = None, time_in: float = None, time_out: float = None, time_total: float = None) → None[source]¶ -
use_tracking
= False¶
-
topic
¶
-
partition
¶
-
offset
¶
-
timestamp
¶
-
timestamp_type
¶
-
headers
¶
-
key
¶
-
value
¶
-
checksum
¶
-
serialized_key_size
¶
-
serialized_value_size
¶
-
acked
¶
-
refcount
¶
-
tp
¶
-
tracked
¶
-
time_in
¶ Monotonic timestamp of when the consumer received this message.
-
time_out
¶ Monotonic timestamp of when the consumer acknowledged this message.
-
time_total
¶ Total processing time (in seconds), or None if the event is still processing.
-
stream_meta
¶ Monitor stores timing information for every stream processing this message here. It’s stored as:
message.stream_meta[id(stream)] = { 'time_in': float, 'time_out': float, 'time_total': float, }
-
classmethod
from_message
(message: Any, tp: faust.types.tuples.TP) → faust.types.tuples.Message[source]¶ - Return type
-
span
¶
-
-
class
faust.types.tuples.
ConsumerMessage
(topic: str, partition: int, offset: int, timestamp: float, timestamp_type: int, headers: Union[List[Tuple[str, bytes]], Mapping[str, bytes], None], key: Optional[bytes], value: Optional[bytes], checksum: Optional[bytes], serialized_key_size: int = None, serialized_value_size: int = None, tp: faust.types.tuples.TP = None, time_in: float = None, time_out: float = None, time_total: float = None) → None[source]¶ Message type used by Kafka Consumer.
-
use_tracking
= True¶
-
-
faust.types.tuples.
tp_set_to_map
(tps: Set[faust.types.tuples.TP]) → MutableMapping[str, Set[faust.types.tuples.TP]][source]¶ - Return type
MutableMapping
[str
,Set
[TP
]]
-
faust.types.tuples.
MessageSentCallback
¶ alias of
typing.Callable
faust.types.web
¶
-
faust.types.web.
ViewHandlerMethod
¶ alias of
typing.Callable
-
faust.types.web.
ViewDecorator
¶ alias of
typing.Callable
-
class
faust.types.web.
ResourceOptions
(*args, **kwargs)[source]¶ CORS Options for specific route, or defaults.
-
allow_credentials
¶ Alias for field number 0
-
expose_headers
¶ Alias for field number 1
-
allow_headers
¶ Alias for field number 2
-
max_age
¶ Alias for field number 3
-
allow_methods
¶ Alias for field number 4
-
-
class
faust.types.web.
CacheBackendT
(app: faust.types.web._AppT, url: Union[yarl.URL, str] = 'memory://', **kwargs) → None[source]¶
-
class
faust.types.web.
CacheT
(timeout: Union[datetime.timedelta, float, str] = None, key_prefix: str = None, backend: Union[Type[faust.types.web.CacheBackendT], str] = None, **kwargs) → None[source]¶
-
class
faust.types.web.
BlueprintT
(*args, **kwargs)[source]¶ -
cache
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, backend: Union[Type[faust.types.web.CacheBackendT], str] = None) → faust.types.web.CacheT[source]¶ - Return type
-
route
(uri: str, *, name: Optional[str] = None, base: Type[faust.types.web.View] = <class 'faust.types.web.View'>) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Type
[View
],Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Type
[View
],Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
static
(uri: str, file_or_directory: Union[str, pathlib.Path], *, name: Optional[str] = None) → None[source]¶ - Return type
None
-
-
faust.types.web.
HttpClientT
¶ alias of
aiohttp.client.ClientSession
Utils¶
faust.utils.codegen
¶
Utilities for generating code at runtime.
-
faust.utils.codegen.
Function
(name: str, args: List[str], body: List[str], *, globals: Dict[str, Any] = None, locals: Dict[str, Any] = None, return_type: Any = <object object>, argsep: str = ', ') → Callable[source]¶ Generate a function from Python.
- Return type
-
faust.utils.codegen.
Method
(name: str, args: List[str], body: List[str], **kwargs) → Callable[source]¶ Generate Python method.
- Return type
-
faust.utils.codegen.
InitMethod
(args: List[str], body: List[str], **kwargs) → Callable[None][source]¶ Generate
__init__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
HashMethod
(attrs: List[str], **kwargs) → Callable[None][source]¶ Generate
__hash__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
EqMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__eq__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
NeMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__ne__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
GeMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__ge__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
GtMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__gt__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
LeMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__le__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
LtMethod
(fields: List[str], **kwargs) → Callable[None][source]¶ Generate
__lt__
method.- Return type
Callable
[[],None
]
-
faust.utils.codegen.
CompareMethod
(name: str, op: str, fields: List[str], **kwargs) → Callable[None][source]¶ Generate object comparison method.
Excellent for
__eq__
,__le__
, etc.Examples
The example:
CompareMethod( name='__eq__', op='==', fields=['x', 'y'], )
Generates a method like this:
def __eq__(self, other): if other.__class__ is self.__class__: return (self.x,self.y) == (other.x,other.y) return NotImplemented
- Return type
Callable
[[],None
]
faust.utils.cron
¶
Crontab Utilities.
faust.utils.functional
¶
Functional utilities.
-
faust.utils.functional.
consecutive_numbers
(it: Iterable[int]) → Iterator[Sequence[int]][source]¶ Find runs of consecutive numbers.
Notes
See https://docs.python.org/2.6/library/itertools.html#examples
faust.utils.json
¶
JSON utilities.
-
faust.utils.json.
str_to_decimal
(s: str, maxlen: int = 1000) → Optional[decimal.Decimal][source]¶ Convert string to
Decimal
.- Parameters
- Raises
ValueError – if length exceeds maximum length, or if value is not a valid number (e.g. Inf, NaN or sNaN).
- Return type
- Returns
Converted number.
- Return type
Decimal
-
class
faust.utils.json.
JSONEncoder
(*, skipkeys=False, ensure_ascii=True, check_circular=True, allow_nan=True, sort_keys=False, indent=None, separators=None, default=None)[source]¶ Faust customized
json.JSONEncoder
.Our version supports additional types like
UUID
, and importantly includes microsecond information in datetimes.-
default
(o: Any, *, sequences: Tuple[type, ...] = (<class 'set'>, ), dates: Tuple[type, ...] = (<class 'datetime.date'>, <class 'datetime.time'>), value_delegate: Tuple[type, ...] = (<enum 'Enum'>, ), has_time: Tuple[type, ...] = (<class 'datetime.datetime'>, <class 'datetime.time'>), _isinstance: Callable = <built-in function isinstance>, _str: Callable = <class 'str'>, _list: Callable = <class 'list'>, textual: Tuple[type, ...] = (<class 'decimal.Decimal'>, <class 'uuid.UUID'>, <class 'bytes'>)) → Any[source]¶ Implement this method in a subclass such that it returns a serializable object for
o
, or calls the base implementation (to raise aTypeError
).For example, to support arbitrary iterators, you could implement default like this:
def default(self, o): try: iterable = iter(o) except TypeError: pass else: return list(iterable) # Let the base class default method raise the TypeError return JSONEncoder.default(self, o)
- Return type
-
-
faust.utils.json.
dumps
(obj: Any, cls: Type[faust.utils.json.JSONEncoder] = <class 'faust.utils.json.JSONEncoder'>, **kwargs) → str[source]¶ Serialize to json. See
json.dumps()
.- Return type
-
faust.utils.json.
loads
(s: str, **kwargs) → Any[source]¶ Deserialize json string. See
json.loads()
.- Return type
faust.utils.platforms
¶
Platform/OS utilities.
faust.utils.tracing
¶
OpenTracing utilities.
-
faust.utils.tracing.
current_span
() → Optional[opentracing.span.Span][source]¶ Get the current span for this context (if any).
- Return type
Optional
[Span
]
-
faust.utils.tracing.
set_current_span
(span: opentracing.span.Span) → None[source]¶ Set the current span for the current context.
- Return type
None
-
faust.utils.tracing.
noop_span
() → opentracing.span.Span[source]¶ Return a span that does nothing when traced.
- Return type
Span
-
faust.utils.tracing.
finish_span
(span: Optional[opentracing.span.Span], *, error: BaseException = None) → None[source]¶ Finish span, and optionally set error tag.
- Return type
None
-
faust.utils.tracing.
operation_name_from_fun
(fun: Any) → str[source]¶ Generate opentracing name from function.
- Return type
faust.utils.urls
¶
URL utilities - Working with URLs.
faust.utils.venusian
¶
Venusian (see venusian).
We define our own interface so we don’t have to specify the callback argument.
-
faust.utils.venusian.
attach
(fun: Callable, category: str, *, callback: Callable[[venusian.Scanner, str, Any], None] = None, **kwargs) → None[source]¶ Shortcut for
venusian.attach()
.This shortcut makes the callback argument optional.
- Return type
None
-
class
faust.utils.venusian.
Scanner
(**kw)[source]¶ -
scan
(package, categories=None, onerror=None, ignore=None)[source]¶ Scan a Python package and any of its subpackages. All top-level objects will be considered; those marked with venusian callback attributes related to
category
will be processed.The
package
argument should be a reference to a Python package or module object.The
categories
argument should be sequence of Venusian callback categories (each category usually a string) or the special valueNone
which means all Venusian callback categories. The default isNone
.The
onerror
argument should either beNone
or a callback function which behaves the same way as theonerror
callback function described in http://docs.python.org/library/pkgutil.html#pkgutil.walk_packages . By default, during a scan, Venusian will propagate all errors that happen during its code importing process, includingImportError
. If you use a customonerror
callback, you can change this behavior.Here’s an example
onerror
callback that ignoresImportError
:import sys def onerror(name): if not issubclass(sys.exc_info()[0], ImportError): raise # reraise the last exception
The
name
passed toonerror
is the module or package dotted name that could not be imported due to an exception.New in version 1.0: the
onerror
callbackThe
ignore
argument allows you to ignore certain modules, packages, or global objects during a scan. It should be a sequence containing strings and/or callables that will be used to match against the full dotted name of each object encountered during a scan. The sequence can contain any of these three types of objects:A string representing a full dotted name. To name an object by dotted name, use a string representing the full dotted name. For example, if you want to ignore the
my.package
package and any of its subobjects or subpackages during the scan, passignore=['my.package']
.A string representing a relative dotted name. To name an object relative to the
package
passed to this method, use a string beginning with a dot. For example, if thepackage
you’ve passed is imported asmy.package
, and you passignore=['.mymodule']
, themy.package.mymodule
mymodule and any of its subobjects or subpackages will be omitted during scan processing.A callable that accepts a full dotted name string of an object as its single positional argument and returns
True
orFalse
. For example, if you want to skip all packages, modules, and global objects with a full dotted path that ends with the word “tests”, you can useignore=[re.compile('tests$').search]
. If the callable returnsTrue
(or anything else truthy), the object is ignored, if it returnsFalse
(or anything else falsy) the object is not ignored. Note that unlike string matches, ignores that use a callable don’t cause submodules and subobjects of a module or package represented by a dotted name to also be ignored, they match individual objects found during a scan, including packages, modules, and global objects.
You can mix and match the three types of strings in the list. For example, if the package being scanned is
my
,ignore=['my.package', '.someothermodule', re.compile('tests$').search]
would causemy.package
(and all its submodules and subobjects) to be ignored,my.someothermodule
to be ignored, and any modules, packages, or global objects found during the scan that have a full dotted name that ends with the wordtests
to be ignored.Note that packages and modules matched by any ignore in the list will not be imported, and their top-level code will not be run as a result.
A string or callable alone can also be passed as
ignore
without a surrounding list.New in version 1.0a3: the
ignore
argument
-
Terminal (TTY) Utilities¶
faust.utils.terminal
¶
-
class
faust.utils.terminal.
Spinner
(file: IO = <_io.TextIOWrapper name='<stderr>' mode='w' encoding='UTF-8'>) → None[source]¶ Progress bar spinner.
-
bell
= '\x08'¶
-
sprites
= ['◢', '◣', '◤', '◥']¶
-
cursor_hide
= '\x1b[?25l'¶
-
cursor_show
= '\x1b[?25h'¶
-
hide_cursor
= True¶
-
stopped
= False¶
-
-
class
faust.utils.terminal.
SpinnerHandler
(spinner: faust.utils.terminal.spinners.Spinner, **kwargs) → None[source]¶ A logger handler that iterates our progress spinner for each log.
-
faust.utils.terminal.
Table
¶ alias of
terminaltables.base_table.BaseTable
-
faust.utils.terminal.
TableDataT
¶ alias of
typing.Sequence
-
faust.utils.terminal.
isatty
(fh: IO) → bool[source]¶ Return True if fh has a controlling terminal.
Notes
Use with e.g.
sys.stdin
.- Return type
-
faust.utils.terminal.
logtable
(data: Sequence[Sequence[str]], *, title: str, target: IO = None, tty: bool = None, headers: Sequence[str] = None, **kwargs) → str[source]¶ Prepare table for logging.
Will use ANSI escape codes if the log file is a tty.
- Return type
-
faust.utils.terminal.
table
(data: Sequence[Sequence[str]], *, title: str, target: IO = None, tty: bool = None, **kwargs) → terminaltables.base_table.BaseTable[source]¶ Create suitable terminaltables table for target.
- Parameters
data (Sequence[Sequence[str]]) – Table data.
target (IO) – Target should be the destination output file for your table, and defaults to
sys.stdout
. ANSI codes will be used if the target has a controlling terminal, but not otherwise, which is why it’s important to pass the correct output file.
- Return type
BaseTable
faust.utils.terminal.spinners
¶
Terminal progress bar spinners.
-
class
faust.utils.terminal.spinners.
Spinner
(file: IO = <_io.TextIOWrapper name='<stderr>' mode='w' encoding='UTF-8'>) → None[source]¶ Progress bar spinner.
-
bell
= '\x08'¶
-
sprites
= ['◢', '◣', '◤', '◥']¶
-
cursor_hide
= '\x1b[?25l'¶
-
cursor_show
= '\x1b[?25h'¶
-
hide_cursor
= True¶
-
stopped
= False¶
-
faust.utils.terminal.tables
¶
Using terminaltables to draw ANSI tables.
-
faust.utils.terminal.tables.
TableDataT
¶ alias of
typing.Sequence
-
faust.utils.terminal.tables.
table
(data: Sequence[Sequence[str]], *, title: str, target: IO = None, tty: bool = None, **kwargs) → terminaltables.base_table.BaseTable[source]¶ Create suitable terminaltables table for target.
- Parameters
data (Sequence[Sequence[str]]) – Table data.
target (IO) – Target should be the destination output file for your table, and defaults to
sys.stdout
. ANSI codes will be used if the target has a controlling terminal, but not otherwise, which is why it’s important to pass the correct output file.
- Return type
BaseTable
-
faust.utils.terminal.tables.
logtable
(data: Sequence[Sequence[str]], *, title: str, target: IO = None, tty: bool = None, headers: Sequence[str] = None, **kwargs) → str[source]¶ Prepare table for logging.
Will use ANSI escape codes if the log file is a tty.
- Return type
-
faust.utils.terminal.tables.
Table
¶ alias of
terminaltables.base_table.BaseTable
Web¶
faust.web.apps.router
¶
HTTP endpoint showing partition routing destinations.
-
class
faust.web.apps.router.
TableList
(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]¶ List routes for all tables.
faust.web.apps.stats
¶
HTTP endpoint showing statistics from the Faust monitor.
faust.web.base
¶
Base interface for Web server and views.
-
class
faust.web.base.
Response
[source]¶ Web server response and status.
-
headers
¶ - Return type
MutableMapping
[~KT, ~VT]
-
-
class
faust.web.base.
BlueprintManager
(initial: Iterable[Tuple[str, Union[_T, str]]] = None) → None[source]¶ Manager of all blueprints.
-
class
faust.web.base.
Web
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Web server and HTTP interface.
-
default_blueprints
= [('/graph', 'faust.web.apps.graph:blueprint'), ('', 'faust.web.apps.stats:blueprint'), ('/router', 'faust.web.apps.router:blueprint'), ('/table', 'faust.web.apps.tables.blueprint')]¶
-
content_separator
= b'\r\n\r\n'¶
-
header_separator
= b'\r\n'¶
-
header_key_value_separator
= b': '¶
-
text
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
html
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
json
(value: Any, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
bytes
(value: bytes, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
route
(pattern: str, handler: Callable, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None) → None[source]¶ - Return type
None
-
add_view
(view_cls: Type[faust.types.web.View], *, prefix: str = '', cors_options: Mapping[str, faust.types.web.ResourceOptions] = None) → faust.types.web.View[source]¶ - Return type
-
url_for
(view_name: str, **kwargs) → str[source]¶ Get URL by view name.
If the provided view name has associated URL parameters, those need to be passed in as kwargs, or a
TypeError
will be raised.- Return type
-
logger
= <Logger faust.web.base (WARNING)>¶
-
faust.web.blueprints
¶
Blueprints define reusable web apps.
They are lazy and need to be registered to an app to be activated:
from faust import web
blueprint = web.Blueprint('users')
cache = blueprint.cache(timeout=300.0)
@blueprint.route('/', name='list')
class UserListView(web.View):
@cache.view()
async def get(self, request: web.Request) -> web.Response:
return web.json(...)
@blueprint.route('/{user_id}/', name='detail')
class UserDetailView(web.View):
@cache.view(timeout=10.0)
async def get(self,
request: web.Request,
user_id: str) -> web.Response:
return web.json(...)
At this point the views are realized and can be used
from Python code, but the cached get
method handlers
cannot be called yet.
To actually use the view from a web server, we need to register the blueprint to an app:
app = faust.App(
'name',
broker='kafka://',
cache='redis://',
)
user_blueprint.register(app, url_prefix='/user/')
At this point the web server will have fully-realized views with actually cached method handlers.
The blueprint is registered with a prefix, so the URL for the
UserListView
is now /user/
, and the URL for the UserDetailView
is /user/{user_id}/
.
Blueprints can be registered to multiple apps at the same time.
-
class
faust.web.blueprints.
Blueprint
(name: str, *, url_prefix: Optional[str] = None) → None[source]¶ Define reusable web application.
-
view_name_separator
= ':'¶
-
cache
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, backend: Union[Type[faust.types.web.CacheBackendT], str] = None) → faust.types.web.CacheT[source]¶ - Return type
-
route
(uri: str, *, name: Optional[str] = None, cors_options: Mapping[str, faust.types.web.ResourceOptions] = None, base: Type[faust.types.web.View] = <class 'faust.types.web.View'>) → Callable[Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Type[faust.types.web.View], Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ - Return type
Callable
[[Union
[Type
[View
],Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Type
[View
],Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
static
(uri: str, file_or_directory: Union[str, pathlib.Path], *, name: Optional[str] = None) → None[source]¶ - Return type
None
-
faust.web.cache
¶
-
class
faust.web.cache.
Cache
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, backend: Union[Type[faust.types.web.CacheBackendT], str] = None, **kwargs) → None[source]¶ Cache interface.
-
ident
= 'faustweb.cache.view'¶
-
view
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, **kwargs) → Callable[Callable, Callable][source]¶ Decorate view to be cached.
-
can_cache_response
(request: faust.types.web.Request, response: faust.types.web.Response) → bool[source]¶ Return
True
for HTTP status codes we CAN cache.- Return type
-
key_for_request
(request: faust.types.web.Request, prefix: str = None, method: str = None, include_headers: bool = False) → str[source]¶ Return a cache key created from web request.
- Return type
-
build_key
(request: faust.types.web.Request, method: str, prefix: str, headers: Mapping[str, str]) → str[source]¶ Build cache key from web request and environment.
- Return type
-
faust.web.cache.backends
¶
Cache backend registry.
faust.web.cache.backends.base
¶
Cache backend - base implementation.
-
class
faust.web.cache.backends.base.
CacheBackend
(app: faust.types.app.AppT, url: Union[yarl.URL, str] = 'memory://', **kwargs) → None[source]¶ Backend for cache operations.
-
logger
= <Logger faust.web.cache.backends.base (WARNING)>¶
-
operational_errors
= ()¶
-
invalidating_errors
= ()¶
-
irrecoverable_errors
= ()¶
-
faust.web.cache.backends.memory
¶
In-memory cache backend.
faust.web.cache.backends.redis
¶
Redis cache backend.
-
class
faust.web.cache.backends.redis.
RedisScheme
[source]¶ Types of Redis configurations.
-
SINGLE_NODE
= 'redis'¶
-
CLUSTER
= 'rediscluster'¶
-
-
class
faust.web.cache.backends.redis.
CacheBackend
(app: faust.types.app.AppT, url: Union[yarl.URL, str], *, connect_timeout: float = None, stream_timeout: float = None, max_connections: int = None, max_connections_per_node: int = None, **kwargs) → None[source]¶ Backend for cache operations using Redis.
faust.web.cache.cache
¶
Cache interface.
-
class
faust.web.cache.cache.
Cache
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, backend: Union[Type[faust.types.web.CacheBackendT], str] = None, **kwargs) → None[source]¶ Cache interface.
-
ident
= 'faustweb.cache.view'¶
-
view
(timeout: Union[datetime.timedelta, float, str] = None, include_headers: bool = False, key_prefix: str = None, **kwargs) → Callable[Callable, Callable][source]¶ Decorate view to be cached.
-
can_cache_response
(request: faust.types.web.Request, response: faust.types.web.Response) → bool[source]¶ Return
True
for HTTP status codes we CAN cache.- Return type
-
key_for_request
(request: faust.types.web.Request, prefix: str = None, method: str = None, include_headers: bool = False) → str[source]¶ Return a cache key created from web request.
- Return type
-
build_key
(request: faust.types.web.Request, method: str, prefix: str, headers: Mapping[str, str]) → str[source]¶ Build cache key from web request and environment.
- Return type
-
faust.web.drivers
¶
Web server driver registry.
faust.web.drivers.aiohttp
¶
Web driver using aiohttp.
-
class
faust.web.drivers.aiohttp.
Web
(app: faust.types.app.AppT, **kwargs) → None[source]¶ Web server and framework implementation using aiohttp.
-
driver_version
= 'aiohttp=3.5.4'¶
-
handler_shutdown_timeout
= 60.0¶
-
cors
¶ - Return type
CorsConfig
-
text
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
html
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
json
(value: Any, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → Any[source]¶ - Return type
-
bytes
(value: bytes, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
route
(pattern: str, handler: Callable, cors_options: Mapping[str, aiohttp_cors.resource_options.ResourceOptions] = None) → None[source]¶ - Return type
None
-
logger
= <Logger faust.web.drivers.aiohttp (WARNING)>¶
-
faust.web.exceptions
¶
HTTP and related errors.
-
exception
faust.web.exceptions.
WebError
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Web related error.
Web related errors will have a status
code
, and adetail
for the human readable error string.It may also keep
extra_context
.-
detail
= 'Default not set on class'¶
-
code
= None¶
-
-
exception
faust.web.exceptions.
ServerError
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Internal Server Error (500).
-
code
= 500¶
-
detail
= 'Internal server error.'¶
-
-
exception
faust.web.exceptions.
ValidationError
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Invalid input in POST data (400).
-
code
= 400¶
-
detail
= 'Invalid input.'¶
-
-
exception
faust.web.exceptions.
ParseError
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Malformed request (400).
-
code
= 400¶
-
detail
= 'Malformed request.'¶
-
-
exception
faust.web.exceptions.
AuthenticationFailed
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Incorrect authentication credentials (401).
-
code
= 401¶
-
detail
= 'Incorrect authentication credentials'¶
-
-
exception
faust.web.exceptions.
NotAuthenticated
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Authentication credentials were not provided (401).
-
code
= 401¶
-
detail
= 'Authentication credentials were not provided.'¶
-
-
exception
faust.web.exceptions.
PermissionDenied
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ No permission to perform action (403).
-
code
= 403¶
-
detail
= 'You do not have permission to perform this action.'¶
-
-
exception
faust.web.exceptions.
NotFound
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Resource not found (404).
-
code
= 404¶
-
detauil
= 'Not found.'¶
-
-
exception
faust.web.exceptions.
MethodNotAllowed
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ HTTP Method not allowed (405).
-
code
= 405¶
-
detail
= 'Method not allowed.'¶
-
-
exception
faust.web.exceptions.
NotAcceptable
(detail: str = None, *, code: int = None, **extra_context) → None[source]¶ Not able to satisfy the request
Accept
header (406).-
code
= 406¶
-
detail
= 'Could not satisfy the request Accept header.'¶
-
faust.web.views
¶
Class-based views.
-
class
faust.web.views.
View
(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]¶ Web view (HTTP endpoint).
-
exception
ServerError
(detail: str = None, *, code: int = None, **extra_context) → None¶ Internal Server Error (500).
-
code
= 500¶
-
detail
= 'Internal server error.'¶
-
-
exception
ValidationError
(detail: str = None, *, code: int = None, **extra_context) → None¶ Invalid input in POST data (400).
-
code
= 400¶
-
detail
= 'Invalid input.'¶
-
-
exception
ParseError
(detail: str = None, *, code: int = None, **extra_context) → None¶ Malformed request (400).
-
code
= 400¶
-
detail
= 'Malformed request.'¶
-
-
exception
NotAuthenticated
(detail: str = None, *, code: int = None, **extra_context) → None¶ Authentication credentials were not provided (401).
-
code
= 401¶
-
detail
= 'Authentication credentials were not provided.'¶
-
-
exception
PermissionDenied
(detail: str = None, *, code: int = None, **extra_context) → None¶ No permission to perform action (403).
-
code
= 403¶
-
detail
= 'You do not have permission to perform this action.'¶
-
-
exception
NotFound
(detail: str = None, *, code: int = None, **extra_context) → None¶ Resource not found (404).
-
code
= 404¶
-
detauil
= 'Not found.'¶
-
-
classmethod
from_handler
(fun: Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]) → Type[faust.web.views.View][source]¶
-
text
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
html
(value: str, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
json
(value: Any, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
bytes
(value: bytes, *, content_type: str = None, status: int = 200, reason: str = None, headers: MutableMapping = None) → faust.web.base.Response[source]¶ - Return type
-
exception
-
faust.web.views.
takes_model
(Model: Type[faust.types.models.ModelT]) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ Decorate view function to return model data.
- Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
-
faust.web.views.
gives_model
(Model: Type[faust.types.models.ModelT]) → Callable[Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]], Union[Callable[[faust.types.web.View, faust.types.web.Request], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]], Callable[[faust.types.web.View, faust.types.web.Request, Any, Any], Union[Coroutine[[Any, Any], faust.types.web.Response], Awaitable[faust.types.web.Response]]]]][source]¶ Decorate view function to automatically decode POST data.
The POST data is decoded using the model you specify.
- Return type
Callable
[[Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]],Union
[Callable
[[View
,Request
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]],Callable
[[View
,Request
,Any
,Any
],Union
[Coroutine
[Any
,Any
,Response
],Awaitable
[Response
]]]]]
CLI¶
faust.cli.agents
¶
Program faust agents
used to list agents.
-
class
faust.cli.agents.
agents
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ List agents.
-
title
= 'Agents'¶
-
headers
= ['name', 'topic', 'help']¶
-
sortkey
= operator.attrgetter('name')¶
-
options
= [<function option.<locals>.decorator>]¶
-
faust.cli.base
¶
Command-line programs using click.
-
faust.cli.base.
argument
(*args, **kwargs) → Callable[Any, Any][source]¶ Create command-line argument.
- SeeAlso:
-
faust.cli.base.
option
(*option_decls, show_default: bool = True, **kwargs) → Callable[Any, Any][source]¶ Create command-line option.
- SeeAlso:
-
faust.cli.base.
find_app
(app: str, *, symbol_by_name: Callable = <function symbol_by_name>, imp: Callable = <function import_from_cwd>) → faust.types.app.AppT[source]¶ Find app by string like
examples.simple
.Notes
This function uses
import_from_cwd
to temporarily add the current working directory toPYTHONPATH
, such that when importing the app it will search the current working directory last.You can think of it as temporarily running with the
PYTHONPATH
set like this:You can disable this with the
imp
keyword argument, for example passingimp=importlib.import_module
.Examples
>>> # If providing the name of a module, it will attempt >>> # to find an attribute name (.app) in that module. >>> # Example below is the same as importing:: >>> # from examples.simple import app >>> find_app('examples.simple')
>>> # If you want an attribute other than .app you can >>> # use : to separate module and attribute. >>> # Examples below is the same as importing:: >>> # from examples.simple import my_app >>> find_app('examples.simple:my_app')
>>> # You can also use period for the module/attribute separator >>> find_app('examples.simple.my_app')
- Return type
AppT
[]
-
class
faust.cli.base.
Command
(ctx: click.core.Context, *args, **kwargs) → None[source]¶ Base class for subcommands.
-
exception
UsageError
(message, ctx=None)¶ An internal exception that signals a usage error. This typically aborts any further handling.
- Parameters
message – the error message to display.
ctx – optionally the context that caused this error. Click will fill in the context automatically in some situations.
-
exit_code
= 2¶
-
show
(file=None)¶
-
abstract
= True¶
-
daemon
= False¶
-
redirect_stdouts
= None¶
-
redirect_stdouts_level
= None¶
-
builtin_options
= [<function version_option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>]¶
-
options
= None¶
-
classmethod
parse
(argv: Sequence[str]) → Mapping[source]¶ Parse command-line arguments in
argv
and return mapping.- Return type
Mapping
[~KT, +VT_co]
-
prog_name
= ''¶
-
as_service
(loop: asyncio.events.AbstractEventLoop, *args, **kwargs) → mode.services.Service[source]¶ - Return type
Service
[]
-
worker_for_service
(service: mode.types.services.ServiceT, loop: asyncio.events.AbstractEventLoop = None) → mode.worker.Worker[source]¶ - Return type
Worker
[]
-
tabulate
(data: Sequence[Sequence[str]], headers: Sequence[str] = None, wrap_last_row: bool = True, title: str = '', title_color: str = 'blue', **kwargs) → str[source]¶ Create an ANSI representation of a table of two-row tuples.
See also
Keyword arguments are forwarded to
terminaltables.SingleTable
Note
If the
--json
option is enabled this returns json instead.- Return type
-
table
(data: Sequence[Sequence[str]], title: str = '', **kwargs) → terminaltables.base_table.BaseTable[source]¶ Format table data as ANSI/ASCII table.
- Return type
BaseTable
-
color
(name: str, text: str) → str[source]¶ Return text having a certain color by name.
- Examples::
>>> self.color('blue', 'text_to_color') >>> self.color('hiblue', text_to_color')
See also
colorclass: for a list of available colors.
- Return type
-
bold_tail
(text: str, *, sep: str = '.') → str[source]¶ Put bold emphasis on the last part of a
foo.bar.baz
string.- Return type
-
say
(message: str, file: IO = None, err: IO = None, **kwargs) → None[source]¶ Print something to stdout (or use
file=stderr
kwarg).Note
Does not do anything if the
--quiet
option is enabled.- Return type
None
-
exception
-
class
faust.cli.base.
AppCommand
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Command that takes
-A app
as argument.-
abstract
= False¶
-
require_app
= True¶
-
value_serialier
= None¶ The codec used to serialize values. Taken from instance parameters or
value_serializer
.
-
classmethod
from_handler
(*options, **kwargs) → Callable[Callable, Type[faust.cli.base.AppCommand]][source]¶ - Return type
Callable
[[Callable
],Type
[AppCommand
]]
-
key_serializer
= None¶ The codec used to serialize keys. Taken from instance parameters or
key_serializer
.
-
to_key
(typ: Optional[str], key: str) → Any[source]¶ Convert command-line argument string to model (key).
- Parameters
Notes
Uses
key_serializer
to set the codec for the key (e.g."json"
), as set by the--key-serializer
option.- Return type
-
to_value
(typ: Optional[str], value: str) → Any[source]¶ Convert command-line argument string to model (value).
- Parameters
Notes
Uses
value_serializer
to set the codec for the value (e.g."json"
), as set by the--value-serializer
option.- Return type
-
to_model
(typ: Optional[str], value: str, serializer: Union[faust.types.codecs.CodecT, str, None]) → Any[source]¶ Convert command-line argument to model.
Generic version of
to_key()
/to_value()
.- Parameters
Notes
Uses
value_serializer
to set the codec for the value (e.g."json"
), as set by the--value-serializer
option.- Return type
-
import_relative_to_app
(attr: str) → Any[source]¶ Import string like “module.Model”, or “Model” to model class.
- Return type
-
to_topic
(entity: str) → Any[source]¶ Convert topic name given on command-line to
app.topic()
.- Return type
-
abbreviate_fqdn
(name: str, *, prefix: str = '') → str[source]¶ Abbreviate fully-qualified Python name, by removing origin.
app.conf.origin
is the package where the app is defined, so if this isexamples.simple
it returns the truncated:>>> app.conf.origin 'examples.simple' >>> abbr_fqdn(app.conf.origin, ... 'examples.simple.Withdrawal', ... prefix='[...]') '[...]Withdrawal'
but if the package is not part of origin it provides the full path:
>>> abbr_fqdn(app.conf.origin, ... 'examples.other.Foo', prefix='[...]') 'examples.other.foo'
- Return type
-
faust.cli.clean_versions
¶
Program faust reset
used to delete local table state.
-
class
faust.cli.clean_versions.
clean_versions
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Delete old version directories.
Warning
This command will result in the destruction of the following files:
Table data for previous versions of the app.
faust.cli.completion
¶
completion - Command line utility for completion.
Supports bash
, ksh
, zsh
, etc.
-
class
faust.cli.completion.
completion
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Output shell completion to be evaluated by the shell.
-
require_app
= False¶
-
faust.cli.faust
¶
Program faust
(umbrella command).
-
class
faust.cli.faust.
agents
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ List agents.
-
title
= 'Agents'¶
-
headers
= ['name', 'topic', 'help']¶
-
sortkey
= operator.attrgetter('name')¶
-
options
= [<function option.<locals>.decorator>]¶
-
-
faust.cli.faust.
call_command
(command: str, args: List[str] = None, stdout: IO = None, stderr: IO = None, side_effects: bool = False, **kwargs) → Tuple[int, IO, IO][source]¶
-
class
faust.cli.faust.
clean_versions
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Delete old version directories.
Warning
This command will result in the destruction of the following files:
Table data for previous versions of the app.
-
class
faust.cli.faust.
completion
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Output shell completion to be evaluated by the shell.
-
require_app
= False¶
-
-
class
faust.cli.faust.
model
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Show model detail.
-
headers
= ['field', 'type', 'default']¶
-
options
= [<function argument.<locals>.decorator>]¶
-
-
class
faust.cli.faust.
models
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ List all available models as a tabulated list.
-
title
= 'Models'¶
-
headers
= ['name', 'help']¶
-
sortkey
= operator.attrgetter('_options.namespace')¶
-
options
= [<function option.<locals>.decorator>]¶
-
-
class
faust.cli.faust.
reset
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Delete local table state.
Warning
This command will result in the destruction of the following files:
The local database directories/files backing tables (does not apply if an in-memory store like memory:// is used).
Notes
This data is technically recoverable from the Kafka cluster (if intact), but it’ll take a long time to get the data back as you need to consume each changelog topic in total.
It’d be faster to copy the data from any standbys that happen to have the topic partitions you require.
-
class
faust.cli.faust.
send
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Send message to agent/topic.
-
options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function argument.<locals>.decorator>, <function argument.<locals>.decorator>]¶
-
coroutine
run
(self, entity: str, value: str, *args, key: str = None, key_type: str = None, key_serializer: str = None, value_type: str = None, value_serializer: str = None, partition: int = 1, timestamp: float = None, repeat: int = 1, min_latency: float = 0.0, max_latency: float = 0.0, **kwargs) → Any[source]¶ - Return type
-
-
class
faust.cli.faust.
tables
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ List available tables.
-
title
= 'Tables'¶
-
-
class
faust.cli.faust.
worker
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Start worker instance for given app.
-
daemon
= True¶
-
redirect_stdouts
= True¶
-
worker_options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>]¶
-
options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>]¶
-
as_service
(loop: asyncio.events.AbstractEventLoop, *args, **kwargs) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
Generate the text banner emitted before the worker starts.
- Return type
-
faust.cli.model
¶
Program faust model
used to list details about a model.
-
class
faust.cli.model.
model
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Show model detail.
-
headers
= ['field', 'type', 'default']¶
-
options
= [<function argument.<locals>.decorator>]¶
-
faust.cli.models
¶
Program faust models
used to list models available.
-
class
faust.cli.models.
models
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ List all available models as a tabulated list.
-
title
= 'Models'¶
-
headers
= ['name', 'help']¶
-
sortkey
= operator.attrgetter('_options.namespace')¶
-
options
= [<function option.<locals>.decorator>]¶
-
faust.cli.params
¶
Python click parameter types.
-
class
faust.cli.params.
CaseInsensitiveChoice
(choices: Iterable[Any])[source]¶ Case-insensitive version of
click.Choice
.
-
class
faust.cli.params.
TCPPort
[source]¶ CLI option: TCP Port (integer in range 1 - 65535).
-
name
= 'range[1-65535]'¶
-
faust.cli.reset
¶
Program faust reset
used to delete local table state.
-
class
faust.cli.reset.
reset
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Delete local table state.
Warning
This command will result in the destruction of the following files:
The local database directories/files backing tables (does not apply if an in-memory store like memory:// is used).
Notes
This data is technically recoverable from the Kafka cluster (if intact), but it’ll take a long time to get the data back as you need to consume each changelog topic in total.
It’d be faster to copy the data from any standbys that happen to have the topic partitions you require.
faust.cli.send
¶
Program faust send
used to send events to agents and topics.
-
class
faust.cli.send.
send
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Send message to agent/topic.
-
options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function argument.<locals>.decorator>, <function argument.<locals>.decorator>]¶
-
coroutine
run
(self, entity: str, value: str, *args, key: str = None, key_type: str = None, key_serializer: str = None, value_type: str = None, value_serializer: str = None, partition: int = 1, timestamp: float = None, repeat: int = 1, min_latency: float = 0.0, max_latency: float = 0.0, **kwargs) → Any[source]¶ - Return type
-
faust.cli.tables
¶
Program faust tables
used to list tables.
faust.cli.worker
¶
Program faust worker
used to start application from console.
-
class
faust.cli.worker.
worker
(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, None] = None, value_serializer: Union[faust.types.codecs.CodecT, str, None] = None, **kwargs) → None[source]¶ Start worker instance for given app.
-
daemon
= True¶
-
redirect_stdouts
= True¶
-
worker_options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>]¶
-
options
= [<function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>, <function option.<locals>.decorator>]¶
-
as_service
(loop: asyncio.events.AbstractEventLoop, *args, **kwargs) → mode.types.services.ServiceT[source]¶ - Return type
ServiceT
[]
Generate the text banner emitted before the worker starts.
- Return type
-
Change history for Faust 1.5¶
This document contain change notes for bugfix releases in the Faust 1.5 series. If you’re looking for previous releases, please visit the History section.
1.6.0¶
- release-date
2019-04-16 5:41 P.M PST
- release-by
Ask Solem (@ask)
This release has minor backward incompatible changes. that only affects those who are using custom sensors. See note below.
Requirements:
Now depends on robinhood-aiokafka 1.0.3
This version disables the “LeaveGroup” timeout added in 1.0.0, as it was causing problems.
Sensors:
on_stream_event_in
now passes state toon_stream_event_out
.This is backwards incompatible but fixes a rare race condition.
Custom sensors that have to use stream_meta must be updated to use this state.
Sensors: Added new sensor methods:
on_rebalance_start(app)
Called when a new rebalance is starting.
on_rebalance_return(app)
Called when the worker has returned data to Kafka.
The next step of the rebalancing phase will be the table recovery process, but this happens in the background and rebalancing will be considered complete for this worker.
on_rebalance_end(app)
Called when all tables are fully recovered and the worker is ready to start processing events in the stream.
Sensors: The type of a sensor that returns/takes state is now Dict instead of a Mapping (as the state is mutable).
Monitor: Optimized latency history cleanup.
Recovery: Fixed bug with highwater returning
None
.Tracing: The
traced
decorator would returnNone
for wrapped coroutines, but we now return the actual return value.- Tracing: Added tracing of aiokafka group coordinator processes
(rebalancing and find coordinator).
1.5.4¶
- release-date
2019-04-9 2:09 P.M PST
- release-by
Ask Solem (@ask)
New
producer_api_version
setting.This can be set to the value “0.10” to remove headers from all messages produced.
Use this if you have downstream consumers that do not support the new Kafka protocol format yet.
The
stream_recovery_delay
setting has been disabled by default.After rebalancing the worker will sleep a bit before starting recovery, the idea being that another recovery may be waiting just behind it so we wait a bit, but this has shown to be not as effective as intended.
Web: Cache can now be configured to take headers into account.
Create the cache manager for your blueprint with the
include_headers
argument:cache = blueprint.cache(timeout=300.0, include_headers=True)
Contributed by Sanyam Satia (@ssatia).
1.5.3¶
- release-date
2019-04-06 11:25 P.M PST
- release-by
Ask Solem (@ask)
Requirements:
Now depends on robinhood-aiokafka 1.0.2
This version disables the “LeaveGroup” timeout added in 1.0.0, as it was causing problems.
Documentation: Fixed spelling.
Tests: Fixed flaky regression test.
1.5.2¶
- release-date
2019-03-28 11:00 A.M PST
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.1.1.
Timers: Prevent drift + add some tiny drift.
Thanks to Bob Haddleton (@bobh66).
App: Autodiscovery now avoids importing
__main__.py
(Issue #324).Added regression test.
The
stream_ack_exceptions
setting has been deprecated.It was not having any effect, and we have no current use for it.
The
stream_ack_cancelled_tasks
setting has been deprecated.It was not having any effect, and we have no current use for it.
App: Autodiscovery failed to load when using
app.main()
in some cases (Issue #323).Added regression test.
Worker: Fixed error during agent shutdown.
Monitor: Monitor assignment latency + assignments completed/failed.
Implemented in the default monitor, but also for statsd and datadog.
CLI: The faust program had the wrong help description.
Docs: Fixes typo in
web_cors_options
example.App: Do no wait for table recovery finished signal, if the app is not starting the recovery service.
1.5.1¶
- release-date
2019-03-24 09:45 P.M PST
- release-by
Ask Solem (@ask)
Fixed hanging in partition assignment introduced in Faust 1.5 (Issue #320).
Contributed by Bob Haddleton (@bobh66).
1.5.0¶
- release-date
2019-03-22 02:18 P.M PST
- release-by
Ask Solem (@ask)
Requirements
Now depends on robinhood-aiokafka 1.0.1
Now depends on Mode 3.1.
Exactly-Once semantics: New
processing_guarantee
setting.Experimental support for “exactly-once” semantics.
This mode ensures tables and counts in tables/windows are consistent even as nodes in the cluster are abruptly terminated.
To enable this mode set the
processing_guarantee
setting:App(processing_guarantee='exactly_once')
Note
If you do enable “exactly_once” for an existing app, you must make sure all workers are running the latest version and possibly starting from a clean set of intermediate topics.
You can accomplish this by bumping up the app version number:
App(version=2, processing_guarantee='exactly_once')
The new processing guarantee require a new version of the assignor protocol, for this reason a “exactly_once” worker will not work with older versions of Faust running in the same consumer group: so to roll out this change you will have to stop all the workers, deploy the new version and only then restart the workers.
New optimizations for stream processing and windows.
If Cython is available during installation, Faust will be installed with compiled extensions.
You can set the
NO_CYTHON
environment variable to disable the use of these extensions even if compiled.New
topic_allow_declare
setting.If disabled your faust worker instances will never actually declare topics.
Use this if your Kafka administrator does not allow you to create topics.
New
ConsumerScheduler
setting.This class can override how events are delivered to agents. The default will go round robin between both topics and partitions, to ensure all topic partitions get a chance of being processed.
Contributed by Miha Troha (@miatroha).
Authentication: Support for GSSAPI authentication.
See documentation for the
broker_credentials
setting.Contributed by Julien Surloppe (@jsurloppe).
Authentication: Support for SASL authentication.
See documentation for the
broker_credentials
setting.New
broker_credentials
setting can also be used to configure SSL authentication.Models: Records can now use comparison operators.
Comparison of models using the
>
,<
,>=
and<=
operators now work similarly todataclasses
.Models: Now raise an error if non-default fields follows default fields.
The following model will now raise an error:
class Account(faust.Record): name: str amount: int = 3 userid: str
This is because a non-default field is defined after a default field, and this would mess up argument ordering.
To define the model without error, make sure you move default fields below any non-default fields:
class Account(faust.Record): name: str userid: str amount: int = 3
Note
Remember that when adding fields to an already existing model you should always add new fields as optional fields.
This will help your application stay backward compatible.
App: Sending messages API now supports a
headers
argument.When sending messages you can now attach arbitrary headers as a dict, or list of tuples; where the values are bytes:
await topic.send(key=key, value=value, headers={'x': b'foo'})
Supported transports
Headers are currently only supported by the default aiokafka transport, and requires Kafka server 0.11 and later.
Agent: RPC operations can now take advantage of message headers.
The default way to attach metadata to values, such as the reply-to address and the correlation id, is to wrap the value in an envelope.
With headers support now landed we can use message headers for this:
@app.agent(use_reply_headers=True) async def x(stream): async for item in stream: yield item ** 2
Faust will be using headers by default in version 2.0.
App: Sending messages API now supports a
timestamp
argument (Issue #276).When sending messages you can now specify the timestamp of the message:
await topic.send(key=key, value=value, timestamp=custom_timestamp)
If no timestamp is provided the current time will be used (
time.time()
).Contributed by Miha Troha (@mihatroha).
App: New
consumer_auto_offset_reset
setting (Issue #267).Contributed by Ryan Whitten (@rwhitten577).
Stream:
group_by
repartitioned topic name now includes the agent name (Issue #284).App: Web server is no longer running in a separate thread by default.
Running the web server in a separate thread is beneficial as it will not be affected by back pressure in the main thread event loop, but it also makes programming harder when it cannot share the loop of the parent.
If you want to run the web server in a separate thread, use the new
web_in_thread
setting.App: New
web_in_thread
controls separate thread for web server.App: New
logging_config
setting.App: Autodiscovery now ignores modules matching “test” (Issue #242).
Contributed by Chris Seto (@chrisseto).
Transport: aiokafka transport now supports headers when using Kafka server versions 0.11 and later.
Tables: New flags can be used to check if actives/standbys are up to date.
app.tables.actives_ready
Set to
True
when tables have synced all active partitions.app.tables.standbys_ready
Set to
True
when standby partitions are up-to-date.
RocksDB: Now crash with
ConsistencyError
if the persisted offset is greater than the current highwater.This means the changelog topic has been modified in Kafka and the recorded offset no longer exists. We crash as we believe this require human intervention, but should some projects have less strict durability requirements we may make this an option.
RocksDB:
len(table)
now only counts databases for active partitions (Issue #270).Agent: Fixes crash when worker assigned no partitions and having the
isolated_partitions
flag enabled (Issue #181).Table: Fixes
KeyError
crash for already removed key.Table: WindowRange is no longer a
NamedTuple
.This will make it easier to avoid hashing mistakes such that window ranges are never represented as both normal tuple and named tuple variants in the table.
Transports: Adds experimental
confluent://
transport.This transport uses the confluent-kafka client.
It is not feature complete, and notably is missing sticky partition assignment so you should not use this transport for tables.
Warning
The
confluent://
transport is not recommended for production use at this time as it has several limitations.Stream: Fixed deadlock when using
Stream.take
to buffer events (Issue #262).Contributed by Nimi Wariboko Jr (@nemosupremo).
Web: Views can now define
options
method to implement a handler for the HTTPOPTIONS
method. (Issue #304)Contributed by Perk Lim (@perklun).
Stream: Fixed acking behavior of
Stream.take
(Issue #266).When
take
is buffering the events should be acked after processing the buffer is complete, instead it was acking when adding into the buffer.Fix contributed by Amit Ripshtos (@amitripshtos).
- Transport: Aiokafka was not limiting how many messages to read in
a fetch request (Issue #292).
Fix contributed by Miha Troha (@mihatroha).
Typing: Added type stubs for
faust.web.Request
.Typing: Fixed type stubs for
@app.agent
decorator.Web: Added support for Cross-Resource Origin Sharing headers (CORS).
See new
web_cors_options
setting.- Debugging: Added OpenTracing hooks to streams/tasks/timers/Crontabs
and rebalancing process.
To enable you have to define a custom
Tracer
class that will record and publish the traces to systems such as Jeager or Zipkin.This class needs to have a
.trace(name, **extra_context)
context manager:from typing import Any, Dict, import opentracing from opentracing.ext.tags import SAMPLING_PRIORITY class FaustTracer: _tracers: Dict[str, opentracing.Tracer] _default_tracer: opentracing.Tracer = None def __init__(self) -> None: self._tracers = {} @cached_property def default_tracer(self) -> opentracing.Tracer: if self._default_tracer is None: self._default_tracer = self.get_tracer('APP_NAME') def trace(self, name: str, sample_rate: float = None, **extra_context: Any) -> opentracing.Span: span = self.default_tracer.start_span( operation_name=name, tags=extra_context, ) if sample_rate is not None: priority = 1 if random.uniform(0, 1) < sample_rate else 0 span.set_tag(SAMPLING_PRIORITY, priority) return span def get_tracer(self, service_name: str) -> opentracing.Tracer: tracer = self._tracers.get(service_name) if tracer is None: tracer = self._tracers[service_name] = CREATE_TRACER(service_name) return tracer._tracer
After implementing the interface you need to set the
app.tracer
attribute:app = faust.App(...) app.tracer = FaustTracer()
That’s it! Now traces will go through your custom tracing implementation.
CLI: Commands
--help
output now always show the default for every parameter.Channels: Fixed bug in
channel.send
that caused a memory leak.This bug was not present when using
app.topic()
.Documentation: Improvements by:
Amit Rip (@amitripshtos).
Sebastian Roll (@SebastianRoll).
Mousse (@zibuyu1995).
Zhanzhao (Deo) Liang (@DeoLeung).
Testing:
99% total unit test coverage
New script to verify documentation defaults are up to date are run for every git commit.
Contributing¶
Welcome!
This document is fairly extensive and you aren’t really expected to study this in detail for small contributions;
The most important rule is that contributing must be easy and that the community is friendly and not nitpicking on details, such as coding style.
If you’re reporting a bug you should read the Reporting bugs section below to ensure that your bug report contains enough information to successfully diagnose the issue, and if you’re contributing code you should try to mimic the conventions you see surrounding the code you’re working on, but in the end all patches will be cleaned up by the person merging the changes so don’t worry too much.
Code of Conduct¶
Everyone interacting in the project’s code bases, issue trackers, chat rooms, and mailing lists is expected to follow the Faust Code of Conduct.
As contributors and maintainers of these projects, and in the interest of fostering an open and welcoming community, we pledge to respect all people who contribute through reporting issues, posting feature requests, updating documentation, submitting pull requests or patches, and other activities.
We are committed to making participation in these projects a harassment-free experience for everyone, regardless of level of experience, gender, gender identity and expression, sexual orientation, disability, personal appearance, body size, race, ethnicity, age, religion, or nationality.
Examples of unacceptable behavior by participants include:
The use of sexualized language or imagery
Personal attacks
Trolling or insulting/derogatory comments
Public or private harassment
Publishing other’s private information, such as physical or electronic addresses, without explicit permission
Other unethical or unprofessional conduct.
Project maintainers have the right and responsibility to remove, edit, or reject comments, commits, code, wiki edits, issues, and other contributions that are not aligned to this Code of Conduct. By adopting this Code of Conduct, project maintainers commit themselves to fairly and consistently applying these principles to every aspect of managing this project. Project maintainers who do not follow or enforce the Code of Conduct may be permanently removed from the project team.
This code of conduct applies both within project spaces and in public spaces when an individual is representing the project or its community.
Instances of abusive, harassing, or otherwise unacceptable behavior may be reported by opening an issue or contacting one or more of the project maintainers.
This Code of Conduct is adapted from the Contributor Covenant, version 1.2.0 available at http://contributor-covenant.org/version/1/2/0/.
Reporting Bugs¶
Security¶
You must never report security related issues, vulnerabilities or bugs
including sensitive information to the bug tracker, or elsewhere in public.
Instead sensitive bugs must be sent by email to security@celeryproject.org
.
If you’d like to submit the information encrypted our PGP key is:
-----BEGIN PGP PUBLIC KEY BLOCK-----
Version: GnuPG v1.4.15 (Darwin)
mQENBFJpWDkBCADFIc9/Fpgse4owLNvsTC7GYfnJL19XO0hnL99sPx+DPbfr+cSE
9wiU+Wp2TfUX7pCLEGrODiEP6ZCZbgtiPgId+JYvMxpP6GXbjiIlHRw1EQNH8RlX
cVxy3rQfVv8PGGiJuyBBjxzvETHW25htVAZ5TI1+CkxmuyyEYqgZN2fNd0wEU19D
+c10G1gSECbCQTCbacLSzdpngAt1Gkrc96r7wGHBBSvDaGDD2pFSkVuTLMbIRrVp
lnKOPMsUijiip2EMr2DvfuXiUIUvaqInTPNWkDynLoh69ib5xC19CSVLONjkKBsr
Pe+qAY29liBatatpXsydY7GIUzyBT3MzgMJlABEBAAG0MUNlbGVyeSBTZWN1cml0
eSBUZWFtIDxzZWN1cml0eUBjZWxlcnlwcm9qZWN0Lm9yZz6JATgEEwECACIFAlJp
WDkCGwMGCwkIBwMCBhUIAgkKCwQWAgMBAh4BAheAAAoJEOArFOUDCicIw1IH/26f
CViDC7/P13jr+srRdjAsWvQztia9HmTlY8cUnbmkR9w6b6j3F2ayw8VhkyFWgYEJ
wtPBv8mHKADiVSFARS+0yGsfCkia5wDSQuIv6XqRlIrXUyqJbmF4NUFTyCZYoh+C
ZiQpN9xGhFPr5QDlMx2izWg1rvWlG1jY2Es1v/xED3AeCOB1eUGvRe/uJHKjGv7J
rj0pFcptZX+WDF22AN235WYwgJM6TrNfSu8sv8vNAQOVnsKcgsqhuwomSGsOfMQj
LFzIn95MKBBU1G5wOs7JtwiV9jefGqJGBO2FAvOVbvPdK/saSnB+7K36dQcIHqms
5hU4Xj0RIJiod5idlRC5AQ0EUmlYOQEIAJs8OwHMkrdcvy9kk2HBVbdqhgAREMKy
gmphDp7prRL9FqSY/dKpCbG0u82zyJypdb7QiaQ5pfPzPpQcd2dIcohkkh7G3E+e
hS2L9AXHpwR26/PzMBXyr2iNnNc4vTksHvGVDxzFnRpka6vbI/hrrZmYNYh9EAiv
uhE54b3/XhXwFgHjZXb9i8hgJ3nsO0pRwvUAM1bRGMbvf8e9F+kqgV0yWYNnh6QL
4Vpl1+epqp2RKPHyNQftbQyrAHXT9kQF9pPlx013MKYaFTADscuAp4T3dy7xmiwS
crqMbZLzfrxfFOsNxTUGE5vmJCcm+mybAtRo4aV6ACohAO9NevMx8pUAEQEAAYkB
HwQYAQIACQUCUmlYOQIbDAAKCRDgKxTlAwonCNFbB/9esir/f7TufE+isNqErzR/
aZKZo2WzZR9c75kbqo6J6DYuUHe6xI0OZ2qZ60iABDEZAiNXGulysFLCiPdatQ8x
8zt3DF9BMkEck54ZvAjpNSern6zfZb1jPYWZq3TKxlTs/GuCgBAuV4i5vDTZ7xK/
aF+OFY5zN7ciZHkqLgMiTZ+RhqRcK6FhVBP/Y7d9NlBOcDBTxxE1ZO1ute6n7guJ
ciw4hfoRk8qNN19szZuq3UU64zpkM2sBsIFM9tGF2FADRxiOaOWZHmIyVZriPFqW
RUwjSjs7jBVNq0Vy4fCu/5+e+XLOUBOoqtM5W7ELt0t1w9tXebtPEetV86in8fU2
=0chn
-----END PGP PUBLIC KEY BLOCK-----
Other bugs¶
Bugs can always be described to the Mailing list, but the best way to report an issue and to ensure a timely response is to use the issue tracker.
Create a GitHub account.
You need to create a GitHub account to be able to create new issues and participate in the discussion.
Determine if your bug is really a bug.
You shouldn’t file a bug if you’re requesting support. For that you can use the Mailing list, or Slack.
Make sure your bug hasn’t already been reported.
Search through the appropriate Issue tracker. If a bug like yours was found, check if you have new information that could be reported to help the developers fix the bug.
Check if you’re using the latest version.
A bug could be fixed by some other improvements and fixes - it might not have an existing report in the bug tracker. Make sure you’re using the latest release of Faust.
Collect information about the bug.
To have the best chance of having a bug fixed, we need to be able to easily reproduce the conditions that caused it. Most of the time this information will be from a Python traceback message, though some bugs might be in design, spelling or other errors on the website/docs/code.
If the error is from a Python traceback, include it in the bug report.
We also need to know what platform you’re running (Windows, macOS, Linux, etc.), the version of your Python interpreter, and the version of Faust, and related packages that you were running when the bug occurred.
If you’re reporting a race condition or a deadlock, tracebacks can be hard to get or might not be that useful. Try to inspect the process to get more diagnostic data. Some ideas:
Include the output from the faust report command:
$ faust -A proj report
This will also include your configuration settings and it try to remove values for keys known to be sensitive, but make sure you also verify the information before submitting so that it doesn’t contain confidential information like API tokens and authentication credentials.
Submit the bug.
By default GitHub will email you to let you know when new comments have been made on your bug. In the event you’ve turned this feature off, you should check back on occasion to ensure you don’t miss any questions a developer trying to fix the bug might ask.
Issue Trackers¶
Bugs for a package in the Faust ecosystem should be reported to the relevant issue tracker.
If you’re unsure of the origin of the bug you can ask the Mailing list, or just use the Faust issue tracker.
Contributors guide to the code base¶
There’s a separate section for internal details, including details about the code base and a style guide.
Read Developer Guide for more!
Versions¶
Version numbers consists of a major version, minor version and a release number. Faust uses the versioning semantics described by SemVer: http://semver.org.
Stable releases are published at PyPI while development releases are only available in the GitHub git repository as tags. All version tags starts with “v”, so version 0.8.0 is the tag v0.8.0.
Branches¶
Current active version branches:
dev (which git calls “master”) (https://github.com/robinhood/faust/tree/master)
You can see the state of any branch by looking at the Changelog:
If the branch is in active development the topmost version info should contain meta-data like:
2.4.0
======
:release-date: TBA
:status: DEVELOPMENT
:branch: dev (git calls this master)
The status
field can be one of:
PLANNING
The branch is currently experimental and in the planning stage.
DEVELOPMENT
The branch is in active development, but the test suite should be passing and the product should be working and possible for users to test.
FROZEN
The branch is frozen, and no more features will be accepted. When a branch is frozen the focus is on testing the version as much as possible before it is released.
dev branch¶
The dev branch (called “master” by git), is where development of the next version happens.
Maintenance branches¶
Maintenance branches are named after the version – for example,
the maintenance branch for the 2.2.x series is named 2.2
.
Previously these were named releaseXX-maint
.
The versions we currently maintain is:
1.0
This is the current series.
Archived branches¶
Archived branches are kept for preserving history only, and theoretically someone could provide patches for these if they depend on a series that’s no longer officially supported.
An archived version is named X.Y-archived
.
Our currently archived branches are:
We don’t currently have any archived branches.
Feature branches¶
Major new features are worked on in dedicated branches. There’s no strict naming requirement for these branches.
Feature branches are removed once they’ve been merged into a release branch.
Tags¶
Tags are used exclusively for tagging releases. A release tag is named with the format
vX.Y.Z
– for examplev2.3.1
.Experimental releases contain an additional identifier
vX.Y.Z-id
– for examplev3.0.0-rc1
.Experimental tags may be removed after the official release.
Working on Features & Patches¶
Note
Contributing to Faust should be as simple as possible, so none of these steps should be considered mandatory.
You can even send in patches by email if that’s your preferred work method. We won’t like you any less, any contribution you make is always appreciated!
However following these steps may make maintainers life easier, and may mean that your changes will be accepted sooner.
Forking and setting up the repository¶
Create your fork¶
First you need to fork the Faust repository, a good introduction to this is in the GitHub Guide: Fork a Repo.
After you have cloned the repository you should checkout your copy to a directory on your machine:
$ git clone git@github.com:username/faust.git
When the repository is cloned enter the directory to set up easy access to upstream changes:
$ cd faust
$ git remote add upstream git://github.com/robinhood/faust.git
$ git fetch upstream
If you need to pull in new changes from upstream you should
always use the --rebase
option to git pull
:
$ git pull --rebase upstream master
With this option you don’t clutter the history with merging commit notes. See Rebasing merge commits in git. If you want to learn more about rebasing see the Rebase section in the GitHub guides.
Start Developing¶
To start developing Faust you should install the requirements and setup the development environment so that Python uses the Faust development directory.
To do so run:
$ make develop
If you want to install requirements manually you should at least install
the git pre-commit hooks (the make develop
command above automatically
runs this as well):
$ make hooks
If you also want to install C extensions, including the RocksDB bindings then you can use make cdevelop instead of make develop:
$ make cdevelop
Note
If you need to work on a different branch than the
one git calls master
, you can
fetch and checkout a remote branch like this:
$ git checkout --track -b 2.0-devel origin/2.0-devel
Running the test suite¶
To run the Faust test suite you need to install a few dependencies.
A complete list of the dependencies needed are located in
requirements/test.txt
.
Both the stable and the development version have testing related dependencies, so install these:
$ pip install -U -r requirements/test.txt
$ pip install -U -r requirements/default.txt
After installing the dependencies required, you can now execute the test suite by calling py.test <pytest:
$ py.test
This will run the unit tests, functional tests and doc example tests, but not integration tests or stress tests.
Some useful options to py.test are:
-x
Stop running the tests at the first test that fails.
-s
Don’t capture output
-v
Run with verbose output.
If you want to run the tests for a single test file only you can do so like this:
$ py.test t/unit/test_app.py
Creating pull requests¶
When your feature/bugfix is complete you may want to submit a pull requests so that it can be reviewed by the maintainers.
Creating pull requests is easy, and also let you track the progress of your contribution. Read the Pull Requests section in the GitHub Guide to learn how this is done.
You can also attach pull requests to existing issues by following the steps outlined here: http://bit.ly/koJoso
Running the tests on all supported Python versions¶
There’s a tox configuration file in the top directory of the distribution.
To run the tests for all supported Python versions simply execute:
$ tox
Use the tox -e
option if you only want to test specific Python versions:
$ tox -e 2.7
Building the documentation¶
To build the documentation you need to install the dependencies
listed in requirements/docs.txt
:
$ pip install -U -r requirements/docs.txt
After these dependencies are installed you should be able to build the docs by running:
$ cd docs
$ rm -rf _build
$ make html
Make sure there are no errors or warnings in the build output.
After building succeeds the documentation is available at _build/html
.
Verifying your contribution¶
To use these tools you need to install a few dependencies. These dependencies
can be found in requirements/dist.txt
.
Installing the dependencies:
$ pip install -U -r requirements/dist.txt
pyflakes & PEP-8¶
To ensure that your changes conform to PEP 8 and to run pyflakes execute:
$ make flakecheck
To not return a negative exit code when this command fails use
the flakes
target instead:
$ make flakes
API reference¶
To make sure that all modules have a corresponding section in the API reference please execute:
$ make apicheck
$ make indexcheck
If files are missing you can add them by copying an existing reference file.
If the module is internal it should be part of the internal reference
located in docs/internals/reference/
. If the module is public
it should be located in docs/reference/
.
For example if reference is missing for the module faust.worker.awesome
and this module is considered part of the public API, use the following steps:
Use an existing file as a template:
$ cd docs/reference/
$ cp faust.schedules.rst faust.worker.awesome.rst
Edit the file using your favorite editor:
$ vim faust.worker.awesome.rst
# change every occurrence of ``faust.schedules`` to
# ``faust.worker.awesome``
Edit the index using your favorite editor:
$ vim index.rst
# Add ``faust.worker.awesome`` to the index.
Commit your changes:
# Add the file to git
$ git add faust.worker.awesome.rst
$ git add index.rst
$ git commit faust.worker.awesome.rst index.rst \
-m "Adds reference for faust.worker.awesome"
Configuration Reference¶
To make sure that all settings have a corresponding section in the configuration reference, please execute:
$ make configcheck
If settings are missing from there an error is produced, and you can proceed
by documenting the settings in docs/userguide/settings.rst
.
Coding Style¶
You should probably be able to pick up the coding style from surrounding code, but it is a good idea to be aware of the following conventions.
We use static types and the mypy type checker to verify them.
Python code must import these static types when using them, so to keep static types lightweight we define interfaces for classes in
faust/types/
.For example for the
fauts.App
class, there is a correspondingfaust.types.app.AppT
; forfaust.Channel
there is afaust.types.channels.ChannelT
and similarly for most other classes in the library.We suffer some duplication because of this, but it keeps static typing imports fast and reduces the need for recursive imports.
In some cases recursive imports still happen, in that case you can “trick” the type checker into importing it, while regular Python does not:
if typing.TYPE_CHECKING: from faust.app import App as _App else: class _App: ... # noqa
Note how we prefix the symbol with underscore to make sure anybody reading the code will think twice before using it.
All Python code must follow the PEP 8 guidelines.
pep8 is a utility you can use to verify that your code is following the conventions.
Docstrings must follow the PEP 257 conventions, and use the following style.
Do this:
def method(self, arg: str) -> None: """Short description. More details. """
or:
def method(self, arg: str) -> None: """Short description."""
but not this:
def method(self, arg: str) -> None: """ Short description. """
Lines shouldn’t exceed 78 columns.
You can enforce this in vim by setting the
textwidth
option:set textwidth=78
If adhering to this limit makes the code less readable, you have one more character to go on. This means 78 is a soft limit, and 79 is the hard limit :)
Import order
Python standard library
Third-party packages.
Other modules from the current package.
or in case of code using Django:
Python standard library (import xxx)
Third-party packages.
Django packages.
Other modules from the current package.
Within these sections the imports should be sorted by module name.
Example:
import threading import time from collections import deque from Queue import Queue, Empty from .platforms import Pidfile from .five import zip_longest, items, range from .utils.time import maybe_timedelta
Wild-card imports must not be used (from xxx import *).
Contributing features requiring additional libraries¶
Some features like a new result backend may require additional libraries that the user must install.
We use setuptools extra_requires for this, and all new optional features that require third-party libraries must be added.
Add a new requirements file in requirements/extras
For the RocksDB store this is
requirements/extras/rocksdb.txt
, and the file looks like this:python-rocksdb
These are pip requirement files so you can have version specifiers and multiple packages are separated by newline. A more complex example could be:
# python-rocksdb 2.0 breaks Foo python-rocksdb>=1.0,<2.0 thrift
Modify
setup.py
After the requirements file is added you need to add it as an option to
setup.py
in theEXTENSIONS
section:EXTENSIONS = { 'debug', 'fast', 'rocksdb', 'uvloop', }
Document the new feature in
docs/includes/installation.txt
You must add your feature to the list in the bundles section of
docs/includes/installation.txt
.After you’ve made changes to this file you need to render the distro
README
file:$ pip install -U requirements/dist.txt $ make readme
Contacts¶
This is a list of people that can be contacted for questions regarding the official git repositories, PyPI packages Read the Docs pages.
If the issue isn’t an emergency then it’s better to report an issue.
Release Procedure¶
Updating the version number¶
The version number must be updated two places:
faust/__init__.py
docs/include/introduction.txt
After you have changed these files you must render
the README
files. There’s a script to convert sphinx syntax
to generic reStructured Text syntax, and the make target readme
does this for you:
$ make readme
Now commit the changes:
$ git commit -a -m "Bumps version to X.Y.Z"
and make a new version tag:
$ git tag vX.Y.Z
$ git push --tags
Releasing¶
Commands to make a new public stable release:
$ make distcheck # checks pep8, autodoc index, runs tests and more
$ make dist # NOTE: Runs git clean -xdf and removes files not in the repo.
$ python setup.py sdist upload --sign --identity='Celery Security Team'
$ python setup.py bdist_wheel upload --sign --identity='Celery Security Team'
If this is a new release series then you also need to do the following:
- Go to the Read The Docs management interface at:
Enter “Edit project”
Change default branch to the branch of this series, for example, use the
1.0
branch for the 1.0 series.Also add the previous version under the “versions” tab.
Developer Guide¶
- Release
1.5
- Date
Apr 17, 2019
Contributors Guide to the Code¶
Module Overview¶
faust.app
Defines the Faust application: configuration, sending messages, etc.
faust.cli
Command-line interface.
faust.exceptions
All custom exceptions are defined in this module.
faust.models
Models describe how message keys and values are serialized/deserialized.
faust.sensors
Sensors record statistics from a running Faust application.
faust.serializers
Serialization using JSON, and codecs for encoding.
faust.stores
Table storage: in-memory, RocksDB, etc.
faust.streams
Stream and table implementation.
faust.topics
Creating topic descriptions, and tools related to topics.
faust.transport
Message transport implementations, e.g. aiokafka.
faust.types
Public interface for static typing.
faust.utils
Utilities. Note: This package is not allowed to import from the top-level package.
faust.web
Web abstractions and web applications served by the Faust web server.
faust.windows
Windowing strategies.
faust.worker
Deployment helper for faust applications: signal handling, graceful shutdown, etc.
Services¶
Everything in Faust that can be started/stopped and restarted, is a
Service
.
Services can start other services, but they can also start asyncio.Task
via self.add_future. These dependencies will be started/stopped/restarted with
the service.
Worker
¶
The worker can be used to start a Faust application, and performs tasks like setting up logging, installs signal handlers and debugging tools etc.
App
¶
The app configures the Faust instance, and is the entry point for just about everything that happens in a Faust instance. Consuming/Producing messages, starting streams and agents, etc.
The app is usually started by Worker
, but can also be started alone if
less operating system interaction is wanted, like if you want to embed Faust
in an application that already sets up signal handling and logging.
Monitor
¶
The monitor is a feature-complete sensor that collects statistics about the running instance. The monitor data can be exposed by the web server.
Producer
¶
The producer is used to publish messages to Kafka topics, and is started whenever necessary. The App will always starts this when a Faust instance is starting, in anticipation of messages to be produced.
Consumer
¶
The Consumer is responsible for consuming messages from Kafka topics, to be
delivered to the streams. It does not actually fetch messages (the
Fetcher
services does that), but it handles everything to do with
consumption, like managing topic subscriptions etc.
Agent
¶
Agents are also services, and any async function decorated using @app.agent
will start with the app.
Conductor
¶
The topic conductor manages topic subscriptions, and forward messages from the Kafka consumer to the streams.
app.stream(topic)
will iterate over the topic: aiter(topic)
.
The conductor feeds messages into that iteration, so the stream
receives messages in the topic:
async for event in stream(event async for event in topic)
TableManager
¶
Manages tables, including recovery from changelog and caching table contents. The table manager also starts the tables themselves, and acts as a registry of tables in the Faust instance.
Store
¶
Every table has a separate store, the store describes how the table is stored in this instance. It could be stored in-memory (default), or as a RocksDB key/value database if the data set is too big to fit in memory.
Partition Assignor¶
Kafka Streams¶
Kafka Streams distributes work across multiple processes by using the consumer group protocol introduced in Kafka 0.9.0. Kafka elects one of the consumers in the consumer group to use its partition assignment strategy to assign partitions to the consumers in the group. The leader gets access to every client’s subscriptions and assigns partitions accordingly.
Kafka Streams uses a sticky partition assignment strategy to minimize movement in the case of rebalancing. Further, it is also redundant in its partition assignment in the sense that it assigns some standby tasks to maintain state store replicas.
The StreamPartitionAssignor used by Kafka Streams works as follows:
Check all repartition source topics and use internal topic manager to make sure they have been created with the right number of partitions.
Using customized partition grouper (DefaultPartitionGrouper) to generate tasks along with their assigned partitions; also make sure that the task’s corresponding changelog topics have been created with the right number of partitions.
Using StickyTaskAssignor to assign tasks to consumer clients.
Assign a task to a client which was running it previously. If there is no such client, assign a task to a client which has its valid local state.
A client may have more than one stream threads. The assignor tries to assign tasks to a client proportionally to the number of threads.
Try not to assign the same set of tasks to two different clients
The assignment is done in one-pass. The result may not satisfy above all.
Within each client, tasks are assigned to consumer clients in round-robin manner.
Faust¶
Faust differs from Kafka Streams in some fundamental ways one of which is that a task in Faust differs from a task in Kafka Streams. Further, Faust doesn’t have the concept of a pre-defined topology and subscribes to streams as and when required in the application.
As a result, the PartitionAssignor
in Faust can get rid of steps one and
two mentioned above and rely on the primitives repartitioning streams and
creating changelog topics to create topics with the correct number of
partitions based on the source topics.
We can largely simplify step three above since there is no concept of task as in Kafka Streams, i.e. we do not introspect the application topology to define a task that would be assigned to the clients. We simply need to make sure that the correct partitions are assigned to the clients and the client streams and processors should handle dealing with the co-partitioning while processing the streams and forwarding data between the different processors.
PartitionGrouper
¶
This can be simplified immensely by grouping the same partition numbers onto the same clients for all topics with the same number of partitions. This way we can guarantee that co-partitioning for all topics requiring co-partitioning (ex: in the case of joins and aggregates) as long as the topics have the correct number of partitions (which we are making the processors implicitly guarantee).
Concerns¶
With the above design we need to be careful around the following concerns:
We need to assign a partition (where changelog) is involved to a client which contains a standby replica for the given topic/partition whenever possible. This can result in unbalanced assignment. We can fix this by evenly and randomly distributing standbys such that over the long term each rebalance will cause the partitions being re-assigned be evenly balanced across all clients.
Network Partitions and other distributed systems failure cases - We delegate this to the Kafka protocol. The Kafka Consumer Protocol handles a lot of the failure conditions involved with the Consumer group leader election such as leader failures, node failures, etc. Network Partitions in Kafka are not handled here as those would result in bigger issues than consumer partition assignment issues.
History¶
This section contains historical change histories, for the latest version please visit Change history for Faust 1.5.
- Release
1.5
- Date
Apr 17, 2019
Change history for Faust 1.4¶
This document contain change notes for bugfix releases in the Faust 1.4.x series. If you’re looking for changes in the latest series, please visit the latest Change history for Faust 1.5.
For even older releases you can visit the History section.
1.4.9¶
- release-date
2019-03-14 04:00 P.M PST
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.0.10.
max_poll_records
accidentally set to 500 by default.The setting has been reverted to its documented default of
None
. This resulted in a 20x performance improvement.CLI: Now correctly returns non-zero exitcode when exception raised inside
@app.command
.CLI: Option
--no_color
renamed to--no-color
to be consistent with other options.This change is backwards compatible and
--no_color
will continue to work.CLI: The
model x
command used “default*” as the field name for default value.$ python examples/withdrawals.py --json model Withdrawal | python -m json.tool [ { "field": "user", "type": "str", "default*": "*" }, { "field": "country", "type": "str", "default*": "*" }, { "field": "amount", "type": "float", "default*": "*" }, { "field": "date", "type": "datetime", "default*": "None" } ]
This now gives “default” without the extraneous star.
App: Can now override the settings class used.
This means you can now easily extend your app with custom settings:
import faust class MySettings(faust.Settings): foobar: int def __init__(self, id: str, *, foobar: int = 0, **kwargs) -> None: super().__init__(id, **kwargs) self.foobar = foobar class App(faust.App): Settings = MySettings app = App('id', foobar=3) print(app.conf.foobar)
1.4.8¶
- release-date
2019-03-11 05:30 P.M PDT
- release-by
Ask Solem (@ask)
Tables: Recovery would hang when changelog have
committed_offset == 0
.Added this test to our manual testing procedure.
1.4.7¶
- release-date
2019-03-08 02:21 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.0.9.
Tables: Read offset not always updated after seek caused recovery to hang.
Consumer: Fix to make sure fetch requests will not block method queue.
App: Fixed deadlock in rebalancing.
Web: Views can now define
options
method to implement a handler for the HTTPOPTIONS
method. (Issue #304)Contributed by Perk Lim (@perklun).
Web: Can now pass headers to HTTP responses.
1.4.6¶
- release-date
2019-01-29 01:52 P.M PDT
- release-by
Ask Solem (@ask)
App: Better support for custom boot strategies by having the app start without waiting for recovery when no tables started.
- Docs: Fixed doc build after intersphinx
URL https://click.palletsprojects.com/en/latest no longer works.
1.4.5¶
- release-date
2019-01-18 02:15 P.M PDT
- release-by
Ask Solem (@ask)
Fixed typo in 1.4.4 release (on_recovery_set_flags -> on_rebalance_start).
1.4.4¶
- release-date
2019-01-18 01:10 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.0.7.
App: App now starts even if there are no agents defined.
Table: Added new flags to detect if actives/standbys are ready.
app.tables.actives_ready
Set to
True
when active tables are recovered from and are ready to use.app.tables.standbys_ready
Set to
True
when standbys are up to date after recovery.
1.4.3¶
- release-date
2019-01-14 03:01 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Require series 0.4.x of robinhood-aiokafka.
Recently version 0.5.0 was released but this has not been tested in production yet, so we have pinned Faust 1.4.x to aiokafka 0.4.x. For more information see Issue #277.
Test requirements now depends on pytest greater than 3.6.
Contributed by Michael Seifert (@seifertm).
Documentation improvements by:
Allison Wang (@allisonwang).
Thibault Serot (@thibserot).
CI: Added CPython 3.7.2 and 3.6.8 to Travis CI build matrix.
1.4.2¶
- release-date
2018-12-19 12:49 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.0.5.
Fixed compatibility with colorlog, thanks to Ryan Whitten (@rwhitten577).
Now compatible with yarl 1.3.x.
Agent: Allow
yield
in agents that useStream.take
(Issue #237).- App: Fixed error “future for different event loop” when web views
send messages to Kafka at startup.
Table: Table views now return HTTP 503 status code during startup when table routing information not available.
App: New
App.BootStrategy
class now decides what services are started when starting the app.Documentation fixes by:
Robert Krzyzanowski (@robertzk).
1.4.1¶
- release-date
2018-12-10 4:49 P.M PDT
- release-by
Ask Solem (@ask)
Web: Disable aiohttp access logs for performance.
1.4.0¶
- release-date
2018-12-07 4:29 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 3.0.
Worker: The Kafka consumer is now running in a separate thread.
The Kafka heartbeat background coroutine sends heartbeats every 3.0 seconds, and if those are missed rebalancing occurs.
This patch moves the aiokafka library inside a separate thread, this way it can send responsive heartbeats and operate even when agents call blocking functions such as
time.sleep(60)
for every event.Table: Experimental support for tables where values are sets.
The new
app.SetTable
constructor creates a table where values are sets. Example uses include keeping track of users at a location:table[location].add(user_id)
.Supports all set operations:
add
,discard
,intersection
,union
,symmetric_difference
,difference
, etc.Sets are kept in memory for fast operation, and this way we also avoid the overhead of constantly serializing/deserializing the data to RocksDB. Instead we periodically flush changes to RocksDB, and populate the sets from disk at worker startup/table recovery.
App: Adds support for Crontab tasks.
You can now define periodic tasks using Cron-syntax:
@app.crontab('*/1 * * * *', on_leader=True) async def publish_every_minute(): print('-- We should send a message every minute --') print(f'Sending message at: {datetime.now()}') msg = Model(random=round(random(), 2)) await tz_unaware_topic.send(value=msg).
See Cron Jobs for more information.
Contributed by Omar Rayward (@omarrayward).
App: Providing multiple URLs to the
broker
setting now works as expected.To facilitate this change
app.conf.broker
is nowList[URL]
instead of a singleURL
.App: New
timezone
setting.This setting is currently used as the default timezone for Crontab tasks.
App: New
broker_request_timeout
setting.Contributed by Martin Maillard (@martinmaillard).
App: New
broker_max_poll_records
setting.Contributed by Alexander Oberegger (@aoberegg).
App: New
consumer_max_fetch_size
setting.Contributed by Matthew Stump (@mstump).
App: New
producer_request_timeout
setting.Controls when producer batch requests expire, and when we give up sending batches as producer requests fail.
This setting has been increased to 20 minutes by default.
Web: aiohttp driver now uses
AppRunner
to start the web server.Contributed by Mattias Karlsson (@stevespark).
Agent: Fixed RPC example (Issue #155).
Contributed by Mattias Karlsson (@stevespark).
Table: Added support for iterating over windowed tables.
See Iterating over keys/values/items in a windowed table..
This requires us to keep a second table for the key index, so support for windowed table iteration requires you to set a
use_index=True
setting for the table:windowed_table = app.Table( 'name', default=int, ).hopping(10, 5, expires=timedelta(minutes=10), key_index=True)
After enabling the
key_index=True
setting you may iterate over keys/items/values in the table:for key in windowed_table.keys(): print(key) for key, value in windowed_table.items(): print(key, value) for value in windowed_table.values(): print(key, value)
The
items
andvalues
views can also select time-relative iteration:for key, value in windowed_table.items().delta(30): print(key, value) for key, value in windowed_table.items().now(): print(key, value) for key, value in windowed_table.items().current(): print(key, value)
- Table: Now raises error if source topic has mismatching
number of partitions with changelog topic. (Issue #137).
Table: Allow using raw serializer in tables.
You can now control the serialization format for changelog tables, using the
key_serializer
andvalue_serializer
keyword arguments toapp.Table(...)
.Contributed by Matthias Wutte (@wuttem).
Worker: Fixed spinner output at shutdown.
Models:
isodates
option now correctly parses timezones without separator such as -0500.Testing: Calling
AgentTestWrapper.put
now propagates exceptions raised in the agent.App: Default value for
stream_recovery_delay
is now 3.0 seconds.CLI: New command “clean_versions” used to delete old version directories (Issue #68).
Web: Added view decorators:
takes_model
andgives_model
.
Change history for Faust 1.3¶
This document contain change notes for bugfix releases in the Faust 1.3.x series. If you’re looking for changes in the latest series, please visit the latest Change history for Faust 1.5.
For even older releases you can visit the History section.
1.3.2¶
- release-date
2018-11-19 1:11 P.M PST
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 2.0.4.
Fixed crash in
perform_seek
when worker was not assigned any partitions.Fixed missing
await
inConsumer.wait_empty
.Fixed hang after rebalance when not using tables.
1.3.1¶
- release-date
2018-11-15 4:12 P.M PST
- release-by
Ask Solem (@ask)
Tables: Fixed problem with table recovery hanging on changelog topics having only a single entry.
1.3.0¶
- release-date
2018-11-08 4:49 P.M PST
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 2.0.3.
Now depends on
robinhood-aiokafka
1.4.19
App: Refactored rebalancing and table recovery (Issue #185).
This optimizes the rebalancing callbacks for greater stability.
Table recovery was completely rewritten to do as little as possible during actual rebalance. This should increase stability and reduce the chance of rebalancing loops.
We no longer attempt to cancel recovery during rebalance, so this should also fix problems with hanging during recovery.
App: Adds new
stream_recovery_delay
setting.In this version we are experimenting with sleeping for 10.0 seconds after rebalance, to allow for more nodes to join/leave before resuming the streams.
This adds some startup delay, but is in general unnoticeable in production.
Windowing: Fixed several edge cases in windowed tables.
Fix contributed by Omar Rayward (@omarrayward).
App: Skip table recovery on rebalance when no tables defined.
RocksDB: Iterating over table keys/items/values now skips standby partitions.
RocksDB: Fixed issue with having “.” in table names (Issue #184).
App: Allow
broker
URL setting without scheme.The default scheme for an URL like “localhost:9092” is
kafka://
.App: Adds
App.on_rebalance_complete
signal.App: Adds
App.on_before_shutdown
signal.Misc: Support for Python 3.8 by importing from collections.abc.
Misc: Got rid of aiohttp deprecation warnings.
Documentation and examples: Improvements contributed by:
Martin Maillard (@martinmaillard).
Omar Rayward (@omarrayward).
Change history for Faust 1.2¶
This document contain change notes for bugfix releases in the Faust 1.2.x series. If you’re looking for changes in the latest series, please visit the latest Change history for Faust 1.5.
For even older releases you can visit the History section.
1.2.2¶
Requirements
Now depends on
aiocontextvars
0.1.x.The new 0.2 version is backwards incompatible and breaks Faust.
Settings: Increases default
broker_session_timeout
to 60.0 seconds.Tables: Fixes use of windowed tables when using
simplejson
.This change makes sure
simplejson
serializestyping.NamedTuple
as lists, and not dictionaries.Fix contributed by Omar Rayward (@omarrayward).
Tables:
windowed_table[key].now()
works outside of stream iteration.Fix contributed by Omar Rayward (@omarrayward).
Examples: New Kubernetes example.
Contributed by Omar Rayward (@omarrayward).
Misc: Fixes
DeprecationWarning
forasyncio.current_task
.Typing: Type checks now compatible with mypy 0.641.
Documentation and examples fixes contributed by
Fabian Neumann (@hellp)
Omar Rayward (@omarrayward)
1.2.1¶
- release-date
2018-10-08 5:00 P.M PDT
- release-by
Ask Solem (@ask)
Worker: Fixed crash introduced in 1.2.0 if no
--loglevel
argument present.Web: The aiohttp driver now exposes
app.web.web_app
attribute.This will be the
aiohttp.web_app.Application
instance used.Documentation: Fixed markup typo in the settings section of the User Guide (Issue #177).
Contributed by Denis Kataev (@kataev).
1.2.0¶
- release-date
2018-10-05 5:23 P.M PDT
- release-by
Ask Solem (@ask).
Fixes¶
- CLI: All commands, including user-defined, now wait for producer to
be fully stopped before shutting down to make sure buffers are flushed (Issue #172).
Table: Delete event in changelog would crash app on table restore (Issue #175)
- App: Channels and topics now take default
key_serializer
/value_serializer
fromkey_type
/value_type
when they are specified as models (Issue #173).This ensures support for custom codecs specified using the model
serializer
class keyword:class X(faust.Record, serializer='msgpack'): x: int y: int
News¶
Requirements
Now depends on Mode 1.18.1.
CLI: Command-line improvements.
All subcommands are now executed by
mode.Worker
.This means all commands will have the same environment set up, including logging, signal handling, blocking detection support, and remote aiomonitor console support.
faust worker
options moved to top level (built-in) options:--logfile
--loglevel
--console-port
--blocking-timeout
To be backwards compatible these options can now appear before and after the
faust worker
command on the command-line (but for all other commands they need to be specified before the command name):$ ./examples/withdrawals.py -l info worker # OK $ ./examples/withdrawals.py worker -l info # OK $ ./examples/withdrawals.py -l info agents # OK $ ./examples/withdrawals.py agents -l info # ERROR!
If you want a long running background command that will run even after returning, use:
daemon=True
.If enabled the program will not shut down until either the user hits Control-c, or the process is terminated by a signal:
@app.command(daemon=True) async def foo(): print('starting') # set up stuff return # command will continue to run after return.
CLI: New
call_command()
utility for testing.This can be used to safely call a command by name, given an argument list.
Producer: New
producer_partitioner
setting (Issue #164)Models: Attempting to instantiate abstract model now raises an error (Issue #168).
App: App will no longer raise if configuration accessed before being finalized.
Instead there’s a new
AlreadyConfiguredWarning
emitted when a configuration key that has been read is modified.- Distribution: Setuptools metadata now moved to
setup.py
to keep in one location.
This also helps the README banner icons show the correct information.
Contributed by Bryant Biggs (@bryantbiggs)
- Distribution: Setuptools metadata now moved to
Documentation and examples improvements by
Denis Kataev (@kataev).
Web Improvements¶
Note
faust.web
is a small web abstraction used by Faust projects.
It is kept separate and is decoupled from stream processing so in the future we can move it to a separate package if necessary.
You can safely disable the web server component of any Faust worker
by passing the --without-web
option.
- Web: Users can now disable the web server from the faust worker
(Issue #167).
Either by passing
faust worker --without-web
on the command-line, or by using the newweb_enable
setting.
Web: Blueprints can now be added to apps by using strings
Example:
app = faust.App('name') app.web.blueprints.add('/users/', 'proj.users.views:blueprint')
Web: Web server can now serve using Unix domain sockets.
The
--web-transport
argument to faust worker, and theweb_transport
setting was added for this purpose.Serve HTTP over Unix domain socket:
faust -A app -l info worker --web-transport=unix:///tmp/faustweb.sock
- Web: Web server is now started by the
App
This makes it easier to access web-related functionality from the app. For example to get the URL for a view by name, you can now use
app.web
to do so after registering a blueprint:app.web.url_for('user:detail', user_id=3)
- Web: Web server is now started by the
New
web
allows you to specify web framework by URL.Default, and only supported web driver is currently
aiohttp://
.View: A view can now define
__post_init__
, just like dataclasses/Faust models can.This is useful for when you don’t want to deal with all the work involved in overriding
__init__
:@blueprint.route('/', name='list') class UserListView(web.View): def __post_init__(self): self.something = True async def get(self, request, response): if self.something: ...
- aiohttp Driver:
json()
response method now uses the Faust json serializer for automatic support of
__json__
callbacks.
- aiohttp Driver:
Web: New cache decorator and cache backends
The cache decorator can be used to cache views, supporting both in-memory and Redis for storing the cache.
from faust import web blueprint = web.Blueprint('users') cache = blueprint.cache(timeout=300.0) @blueprint.route('/', name='list') class UserListView(web.View): @cache.view() async def get(self, request: web.Request) -> web.Response: return web.json(...) @blueprint.route('/{user_id}/', name='detail') class UserDetailView(web.View): @cache.view(timeout=10.0) async def get(self, request: web.Request, user_id: str) -> web.Response: return web.json(...)
At this point the views are realized and can be used from Python code, but the cached
get
method handlers cannot be called yet.To actually use the view from a web server, we need to register the blueprint to an app:
app = faust.App( 'name', broker='kafka://', cache='redis://', ) app.web.blueprints.add('/user/', 'where.is:user_blueprint')
After this the web server will have fully-realized views with actually cached method handlers.
The blueprint is registered with a prefix, so the URL for the
UserListView
is now/user/
, and the URL for theUserDetailView
is/user/{user_id}/
.
Change history for Faust 1.1¶
This document contain change notes for bugfix releases in the Faust 1.1.x series. If you’re looking for changes in the latest series, please visit the latest Change history for Faust 1.5.
For even older releases you can visit the History section.
1.1.3¶
- release-date
2018-09-21 4:23 P.M PDT
- release-by
Ask Solem (@ask)
Producer: Producing messages is now 8x to 20x faster.
- Stream: The
stream_publish_on_commit
setting is now disabled by default.
Some agents produce data into topics: they forward data after processing or modify tables requiring changelog events to be sent.
Kafka’s at-least-once delivery guarantee means we will never lose a message, and we can be certain any event sent to the source topic will be processed. It also means any source event can be processed multiple times.
If the source event is processed many times and part of the agents processing includes forwarding that event, or producing a new kind of event, then that will also happen as many times as the source event is reprocessed.
The
stream_publish_on_commit
setting attempts to minimize the chances of duplicate messages being produced, by buffering up any events sent in the agent and holding on to it until the offset of the source event is committed.Here’s an agent forwarding values to another topic:
@app.agent(source_topic) async def forward(stream): async for value in stream: await destination_topic.send(value=value)
If we execute this with
stream_publish_on_commit
enabled, then the send operation will be delayed until we have committed the offset for the source event.This works well when we commit often, but completely falls apart if the buffer grows too large and we have too much to do during commit.
The commit operation works like this (in pseudo code) when
stream_publish_on_commit
is enabled:async def commit(self): committable_offsets: Dict[TopicPartition, int] = ... # Operation A (send buffered messages related to source offsets) for tp, offset in committable_offsets.items(): send_messages_buffered_up_until_offset(tp, offset) # Operation B (actually tell Kafka the new offsets) consumer.commit(committable_offsets)
This is not an atomic operation - the worker could crash between completing Operation A and Operation B. If there are 1000 messages to send, it could send 500 of them then crash without committing.
In this case we end up with 500 duplicate messages when the source offsets are reprocessed. Is this safer than producing one and one, and committing fast? Probably not.
That said, if you make sure the buffer never grows too large then you can take advantage of this setting to actually reduce the number of duplicate messages sent when a source topic is reprocessed.
If you want to experiment with this, tweak the
broker_commit_every
andbroker_commit_interval
settings:app = faust.App('name', broker_commit_every=100, broker_commit_interval=1.0, stream_publish_on_commit=True)
The good news is that Kafka transactions are on the horizon. As soon as we have support in a Python client, we can perform this atomically, and without the overhead of buffering up messages until commit time (note from future: “exactly-once” was implemented in Faust 1.5).
- Stream: The
1.1.2¶
- release-date
2018-09-19 5:09 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 1.17.3.
Agent: Agents having concurrency=n was executing events n times.
An unrelated change caused these additional actors to have separate channels, when they should share the same channel.
The only tests verifying this was using mocks, so we’ve added a new functional test in
t/functional/agents
to be sure it won’t happen again.This test also demonstrated a case of starvation when using concurrency: a single concurrency slot could starve others from doing work. To fix this a
sleep(0)
was added toStream.__aiter__
, this could improve performance in general for workers with many agents.Huge thanks to Zhy on the Faust slack channel for testing and identifying this issue.
Agent: Less logging noise when using
concurrency
.This removes the additionally emitted “Starting…”/”Stopping…” logs, especially noisy with
@app.agent(concurrency=1000)
.
1.1.1¶
- release-date
2018-09-17 4:06 P.M PDT
- release-by
Ask Solem (@ask)
Requirements
Now depends on Mode 1.17.2.
- Web: Blueprint registered to app with URL prefix would end up
having double-slash.
- Documentation: Added project layout suggestions
to the application user guide.
Types: annotations now passing checks on mypy 0.630.
1.1.0¶
- release-date
2018-09-14 1:07 P.M PDT
- release-by
Ask Solem (@ask)
Important Notes¶
API: Agent/Channel.send now support keyword-only arguments only
Users often make the mistake of doing:
channel.send(x)
and expect that to send
x
as the value.But the signature is
(key, value, ...)
, so it ends up beingchannel.send(key=x, value=None)
.Fixing this will come in two parts:
Faust 1.1 (this change): Make them keyword-only arguments
This will make it an error if the names of arguments are not specified:
channel.send(key, value)
Needs to be changed to:
channel.send(key=key, value=value)
- Faust 1.2: We will change the signature
to
channel.send(value, key=key, ...)
At this stage all existing code will have changed to using keyword-only arguments.
App: The default key serializer is now
raw
(Issue #142).The default value serializer will still be
json
, but for keys it does not make as much sense to use json as the default: keys are very rarely expressed using complex structures.If you depend on the Faust 1.0 behavior you should override the default key serializer for the app:
app = faust.App('myapp', ..., key_serializer='json')
Contributed by Allison Wang (@allisonwang)
No longer depends on click_completion
If you want to use the shell completion command, you now have to install that dependency locally first:
$ ./examples/withdrawals.py completion Usage: withdrawals.py completion [OPTIONS] Error: Missing required dependency, but this is easy to fix. Run `pip install click_completion` from your virtualenv and try again!
Installing click_completion:
$ pip install click_completion [...]
News¶
Requirements
Now depends on Mode 1.17.1.
No longer depends on click_completion
Now works with CPython 3.6.0.
Models: Record: Now supports decimals option to convert string decimals back to Decimal
This can be used for any model to enable “Decimal-fields”:
class Fundamentals(faust.Record, decimals=True): open: Decimal high: Decimal low: Decimal volume: Decimal
When serialized this model will use string for decimal fields (the Javascript float type cannot be used without losing precision, it is a float after all), but when deserializing Faust will reconstruct them as Decimal objects from that string.
Model: Records now support custom coercion handlers.
Coercion converts one type into another, for example from string to
datettime
, or int/string toDecimal
.In models this means conversion from the serialized form back into a corresponding Python type.
To define a model where all
UUID
fields are serialized to string, but then converted back toUUID
objects when deserialized, do this:from uuid import UUID import faust class Account(faust.Record, coercions={UUID: UUID}): id: UUID
What about non-json serializable types?
The use of UUID in this example leaves one important detail out: json doesn’t support this type so how can models serialize it?
The Faust JSON serializer adds support for UUID objects by default, but if you have a custom class you would need to add that capability by adding a
__json__
handler:class MyType: def __init__(self, value: str): self.value = value def __json__(self): return self.value
You’d get tired writing this out for every class, so why not make an abstract model subclass:
from uuid import UUID import faust class UUIDAwareRecord(faust.Record, abstract=True, coercions={UUID: UUID}): ... class Account(UUIDAwareRecord): id: UUID
App: New
ssl_context
adds authentication support to Kafka.Contributed by Mika Eloranta (@melor).
Monitor: New Datadog monitor (Issue #160)
Contributed by Allison Wang (@allisonwang).
- App:
@app.task
decorator now acceptson_leader
argument (Issue #131).
Tasks created using the
@app.task
decorator will run once a worker is fully started.Similar to the
@app.timer
decorator, you can now create one-shot tasks that run on the leader worker only:@app.task(on_leader=True) async def mytask(): print('WORKER STARTED, AND I AM THE LEADER')
The decorated function may also accept the
app
as an argument:@app.task(on_leader=True) async def mytask(app): print(f'WORKER FOR APP {app} STARTED, AND I AM THE LEADER')
- App:
App: New
app.producer_only
attribute.If set the worker will start the app without consumer/tables/agents/topics.
App:
app.http_client
property is now read-write.Channel: In-memory channels were not working as expected.
Channel.send(key=key, value=value)
now works as expected.app.channel()
accidentally set themaxsize
to 1 by default, creating a deadlock.Channel.send()
now disregards thestream_publish_on_commit
setting.
Transport: aiokafka: Support timestamp-less messages
Fixes error when data sent with old Kafka broker not supporting timestamps:
[2018-08-27 08:00:49,262: ERROR]: [^--Consumer]: Drain messages raised: TypeError("unsupported operand type(s) for /: 'NoneType' and 'float'",) Traceback (most recent call last): File "faust/transport/consumer.py", line 497, in _drain_messages async for tp, message in ait: File "faust/transport/drivers/aiokafka.py", line 449, in getmany record.timestamp / 1000.0, TypeError: unsupported operand type(s) for /: 'NoneType' and 'float'
Contributed by Mika Eloranta (@melor).
Distribution:
pip install faust
no longer installs the examples directory.Fix contributed by Michael Seifert (@seifertm)
Web: Adds exception handling to views.
A view can now bail out early via raise self.NotFound() for example.
Web:
@table_route
decorator now supports taking key from the URL path.This is now used in the
examples/word_count.py
example to add an endpoint/count/{word}/
that routes to the correct worker with that count:@app.page('/word/{word}/count/') @table_route(table=word_counts, match_info='word') async def get_count(web, request, word): return web.json({ word: word_counts[word] })
Web: Support reverse lookup from view name via
url_for
web.url_for(view_name, **params)
Web: Adds support for Flask-like “blueprints”
Blueprint is basically just a description of a reusable app that you can add to your web application.
Blueprints are commonly used in most Flask-like web frameworks, but Flask blueprints are not compatible with e.g. Sanic blueprints.
The Faust blueprint is not directly compatible with any of them, but that should be fine.
To define a blueprint:
from faust import web blueprint = web.Blueprint('user') @blueprint.route('/', name='list') class UserListView(web.View): async def get(self, request: web.Request) -> web.Response: return self.json({'hello': 'world'}) @blueprint.route('/{username}/', name='detail') class UserDetailView(web.View): async def get(self, request: web.Request) -> web.Response: name = request.match_info['username'] return self.json({'hello': name}) async def post(self, request: web.Request) -> web.Response: ... async def delete(self, request: web.Request) -> web.Response: ...
Then to add the blueprint to a Faust app you register it:
blueprint.register(app, url_prefix='/users/')
Note
You can also create views from functions (in this case it will only support GET):
@blueprint.route('/', name='index') async def hello(self, request): return self.text('Hello world')
Why?
Asyncio web frameworks are moving quickly, and we want to be able to quickly experiment with different backend drivers.
Blueprints is a tiny abstraction that fit well into the already small web abstraction that we do have.
Documentation and examples improvements by
Tom Forbes (@orf).
Matthew Grossman (@matthewgrossman)
Denis Kataev (@kataev)
Allison Wang (@allisonwang)
Huyuumi (@diplozoon)
Project¶
CI: The following Python versions have been added to the build matrix:
CPython 3.7.0
CPython 3.6.6
CPython 3.6.0
Git:
All the version tags have been cleaned up to follow the format
v1.2.3
.New active maintenance branches:
1.0
and1.1
.
Change history for Faust 1.0¶
This document contain change notes for bugfix releases in the Faust 1.x series. If you’re looking for changes in the latest series, please visit the latest Change history for Faust 1.5.
For even older releases you can visit the History section.
1.0.30¶
- release-date
2018-08-15 3:17 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on Mode 1.15.1.
- Typing:
faust.types.Message.timestamp_type
is now the correct int
, previously it was string by message.
- Typing:
Models: Records can now have recursive fields.
For example a tree structure model having a field that refers back to itself:
class Node(faust.Record): data: Any children: List['Node']
- Models: A field of type
List[Model]
no longer raises an exception if the value provided is
None
.
- Models: A field of type
Models: Adds support for
--strict-optional
-style fields.Previously the following would work:
class Order(Record): account: Account = None
The account is considered optional from a typing point of view, but only if the mypy option
--strict-optional
is disabled.Now that
--strict-optional
is enabled by default in mypy, this version adds support for fields such as:class Order(Record): account: Optional[Account] = None history: Optional[List[OrderStatus]]
- Models: Class options such as
isodates
/include_metadata
/etc. are now inherited from parent class.
- Models: Class options such as
Stream: Fixed
NameError
when pushing non-Event value into stream.
1.0.29¶
- release-date
2018-08-10 5:00 P.M PDT
- release-by
Vineet Goel
Requirements
Now depends on robinhood-aiokafka 0.4.18
The coordination routine now ensures the program stops when receiving a
aiokafka.errors.UnknownError
from the Kafka broker. This leaves recovery up to the supervisor.
Table: Fixed hanging at startup/rebalance on Python 3.7 (Issue #134).
Workaround for
asyncio
bug seemingly introduced in Python 3.7, that left the worker hanging at startup when attempting to recover a table without any data.Monitor: More efficient updating of highwater metrics (Issue #139).
Partition Assignor: The assignor now compresses the metadata being passed around to all application instances for efficiency and to avoid extreme cases where the metadata is too big.
1.0.28¶
- release-date
2018-08-08 11:25 P.M PDT
- release-by
Vineet Goel
Monitor: Adds consumer stats such as last read offsets, last committed offsets and log end offsets to the monitor. Also added to the StatsdMonitor.
aiokafka: Changes how topics are created to make it more efficient. We now are smarter about finding kafka cluster controller instead of trial and error.
Documentation: Fixed links to Slack and other minor fixes.
1.0.27¶
- release-date
2018-07-30 04:00 P.M PDT
- release-by
Ask Solem
No code changes
Fixed links to documentation in README.rst
1.0.25¶
- release-date
2018-07-27 12:43 P.M PDT
- release-by
Ask Solem
stream_publish_on_commit
accidentally disabled by default.This made the rate of producing much slower, as the default buffering settings are not optimized.
The
App.rebalancing
flag is now reset after the tables have recovered.
1.0.24¶
- release-date
2018-07-12 6:54 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on robinhood-aiokafka 0.4.17
This fixed an issue where the consumer would be left hanging without a connection to Kafka.
1.0.23¶
- release-date
2018-07-11 5:00 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on robinhood-aiokafka 0.4.16
Now compatible with Python 3.7.
Setting
stream_wait_empty
is now disabled by default (Issue #117).Documentation build now compatible with Python 3.7.
Fixed
ForwardRef has no attribute __origin__
error.Fixed
DeprecatedInSphinx2.0
warnings.
Web: Adds
app.on_webserver_init(web)
callback for ability to serve static files usingweb.add_static
.Web: Adds
web.add_static(prefix, fs_path)
Worker: New
App.unassigned
attribute is now set if the worker does not have any assigned partitions.CLI: Console colors was disabled by default.
1.0.22¶
- release-date
2018-06-27 5:35 P.M PDT
- release-by
Vineet Goel
aiokafka: Timeout for topic creation now wraps entire topic creation. Earlier this timeout was for each individual request.
testing: Added stress testing suite.
1.0.21¶
- release-date
2018-06-27 1:43 P.M PDT
- release-by
Ask Solem
Warning
This changes the package name of kafka
to rhkafka
.
Requirements
Now depends on robinhood-aiokafka 0.4.14
Now depends on Mode 1.15.0.
1.0.20¶
- release-date
2018-06-26 2:35 P.M PDT
- release-by
Vineet Goel
Monitor: Added
Monitor.count
to add arbitrary metrics to app monitor.Statsd Monitor: Normalize agent metrics by removing memory address to avoid spamming statsd with thousands of unique metrics per agent.
1.0.19¶
- release-date
2018-06-25 6:40 P.M PDT
- release-by
Vineet Goel
Assignor: Fixed crash if initial state of assignment is invalid. This was causing the following error:
ValueError('Actives and Standbys are disjoint',).
during partition assignment.
1.0.18¶
- release-date
2018-06-21 3:53 P.M PDT
- release-by
Ask Solem
Worker: Fixed
KeyError: TopicPartition(topic='...', partition=x)
occurring during rebalance.
1.0.17¶
- release-date
2018-06-21 3:15 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on robinhood-aiokafka 0.4.13
We now raise an error if the official aiokafka or kafka-python is installed.
Faust depends on a fork of aiokafka and can not be installed with the official versions of aiokafka and kafka-python.
If you have those in requirements, please remove them from your
virtualenv
and remove them from requirements.Worker: Fixes hanging in wait_empty.
This should also make rebalances faster.
Worker: Adds timeout on topic creation.
1.0.16¶
- release-date
2018-06-19 3:46 P.M PDT
- release-by
Ask Solem
- Worker: aiokafka create topic request default timeout now set
to 20 seconds (previously it was accidentally set to 1000 seconds).
- Worker: Fixes crash from
AssertionError
wheretable._revivers
is an empty list.
- Worker: Fixes crash from
Distribution: Adds
t/misc/scripts/rebalance/killer-always-same-node.sh
.
1.0.15¶
- release-date
2018-06-14 7:36 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on robinhood-aiokafka 0.4.12
Worker: Fixed problem where worker does not recover after MacBook sleeping and waking up.
Worker: Fixed crash that could lead to rebalancing loop.
Worker: Removed some noisy errors that weren’t really errors.
1.0.14¶
- release-date
2018-06-13 5:58 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on robinhood-aiokafka 0.4.11
Worker: aiokafka’s heartbeat thread would sometimes keep the worker alive even though the worker was trying to shutdown.
An error could have happened many hours ago causing the worker to crash and attempt a shutdown, but then the heartbeat thread kept the worker from terminating.
Now the rebalance will check if the worker is stopped and then appropriately stop the heartbeat thread.
Worker: Fixed error that caused rebalancing to hang:
"ValueError: Set of coroutines/Futures is empty."
.Worker: Fixed error “Coroutine x tried to break fence owned by y”
This was added as an assertion to see if multiple threads would use the variable at the same time.
Worker: Removed logged error “not assigned to topics” now that we automatically recover from non-existing topics.
Tables: Ignore
asyncio.CancelledError
while stopping standbys.Distribution: Added scripts to help stress test rebalancing in
t/misc/scripts/rebalance
.
1.0.13¶
- release-date
2018-06-12 2:10 P.M PDT
- release-by
Ask Solem
Worker: The Kafka fetcher service was taking too long to shutdown on rebalance.
If this takes longer than the session timeout, it triggers another rebalance, and if it happens repeatedly this will cause the cluster to be in a state of constant rebalancing.
Now we use future cancellation to stop the service as fast as possible.
Worker: Fetcher was accidentally started too early.
This didn’t lead to any problems that we know of, but made the start a bit slower than it needs to.
Worker: Fixed race condition where partitions were paused while fetching from them.
Worker: Fixed theoretical race condition hang if web server started and stopped in quick succession.
Statsd: The statsd monitor prematurely initialized the event loop on module import.
We had a fix for this, but somehow forgot to remove the “hard coded super” that was set to call:
Service.__init__(self, **kwargs)
.The class is not even a subclass of Service anymore, and we are lucky it manifests merely when doing something drastic, like
py.test
, recursively importing all modules in a directory.
1.0.12¶
- release-date
2018-06-06 1:34 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on Mode 1.14.1.
Worker: Producer crashing no longer causes the consumer to hang at shutdown while trying to publish attached messages.
1.0.11¶
- release-date
2018-05-31 16:41 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on Mode 1.13.0.
Now depends on robinhood-aiokafka
We have forked aiokafka to fix some issues.
Now handles missing topics automatically, so you don’t have to restart the worker the first time when topics are missing.
Mode now registers as a library having static type annotations.
This conforms to PEP 561 – a new specification that defines how Python libraries register type stubs to make them available for use with static analyzers like mypy and pyre-check.
Typing: Faust codebase now passes
--strict-optional
.Settings: Added new settings
- Aiokafka: Removes need for consumer partitions lock: this fixes
rare deadlock.
Worker: Worker no longer hangs for few minutes when there is an error.
1.0.10¶
- release-date
2018-05-15 16:02 P.M PDT
- release-by
Vineet Goel
Worker: Stop reading changelog when no remaining messages.
1.0.9¶
- release-date
2018-05-15 15:42 P.M PDT
- release-by
Vineet Goel
Worker: Do not stop reading standby updates.
1.0.8¶
- release-date
2018-05-15 11:00 A.M PDT
- release-by
Vineet Goel
Tables
Fixes bug due to which we were serializing
None
values while recording a key delete to the changelog. This was causing the deleted keys to never be deleted from the changelog.We were earlier not persisting offsets of messages read during changelog reading (or standby recovery). This would cause longer recovery times if recovery was ever interrupted.
App: Added flight recorder for consumer group rebalances for debugging.
1.0.7¶
- release-date
2018-05-14 4:53 P.M PDT
- release-by
Ask Solem
Requirements
Now depends on Mode 1.12.5.
App:
key_type
andvalue_type
can now be set to:int
: key/value is number stored as stringfloat
: key/value is floating point number stored as string.decimal.Decimal
key/value is decimal stored as string.
Agent: Fixed support for
group_by
/through
after change to reuse the same stream after agent crashing.Agent: Fixed
isolated_partitions=True
after change in v1.0.3.Initialization of the agent-by-topic index was in 1.0.3 moved to the
AgentManager.start
method, but it turns outAgentManager
is a regular class, and not a service.AgentManager
is now a service responsible for starting/stopping the agents required by the app.Agent: Include active partitions in repr when
isolated_partitions=True
.Agent: Removed extraneous ‘agent crashed’ exception in logs.
CLI: Fixed autodiscovery of commands when using
faust -A app
.Consumer: Appropriately handle closed fetcher.
New shortcut:
faust.uuid()
generates UUID4 ids as string.
1.0.6¶
- release-date
2018-05-11 11:15 A.M PDT
- release-by
Vineet Goel
Requirements:
Now depends on Aiokafka 0.4.7.
Table: Delete keys when raw value in changelog set to
None
This was resulting in deleted keys still being present with value
None
upon recovery.Transports: Crash app on CommitFailedError thrown by aiokafka.
App would get into a weird state upon a commit failed error thrown by the consumer thread in the aiokafka driver.
1.0.5¶
- release-date
2018-05-08 4:09 P.M PDT
- release-by
Ask Solem
Requirements:
Now depends on Mode 1.12.4.
Agents: Fixed problem with hanging after agent raises exception.
If an agent raises an exception we cannot handle it within the stream iteration, so we need to restart the agent.
Starting from this change, even though we restart the agent, we reuse the same
faust.Stream
object that the crashed agent was using.This makes recovery more seamless and there are fewer steps involved.
Transports: Fixed worker hanging issue introduced in 1.0.4.
In version 1.0.4 we introduced a bug in the round-robin scheduling of topic partitions that manifested itself by hanging with 100% CPU usage.
After processing all records in all topic partitions, the worker would spin loop.
API: Added new base class for windows:
faust.Window
There was the typing interface
faust.types.windows.WindowT
, but now there is also a concrete base class that can be used in for exampleMock(autospec=Window)
.Tests: Now takes advantage of the new
AsyncMock
.
1.0.4¶
- release-date
2018-05-08 11:45 A.M PDT
- release-by
Vineet Goel
Transports:
In version-1.0.2 we implemented fair scheduling in aiokafka transport such that while processing the worker had an equal chance of processing each assigned Topic. Now we also round-robin through topic partitions within topics such that the worker has an equal chance of processing message from each assigned partition within a topic as well.
1.0.3¶
- release-date
2018-05-07 3:45 P.M PDT
- release-by
Ask Solem
Tests:
Adds 5650 lines of tests, increasing test coverage to 90%.
Requirements:
Now depends on Mode 1.12.3.
Development:
CI now builds coverage.
CI now tests multiple CPython versions:
CPython 3.6.0
CPython 3.6.1
CPython 3.6.2
CPython 3.6.3
CPython 3.6.4
CPython 3.6.5
Backward incompatible changes:
Removed
faust.Set
unused by any internal applications.
Fixes:
app.agents
did not forward app toAgentManager
.The agent manager does not use the app, but fixing this in anticipation of people writing custom agent managers.
AgentManager
: On partitions revokedthe agent manager now makes sure there’s only one call to each agents
agent.on_partitions_revoked
callback.This is more of a pedantic change, but could have caused problems for advanced topic configurations.
1.0.2¶
- release-date
2018-05-03 3:32 P.M PDT
- release-by
Ask Solem
Transports: Implements fair scheduling in aiokafka transport.
We now round-robin through topics when processing fetched records from Kafka. This helps us avoid starvation when some topics have many more records than others, and also takes into account that different topics may have wildly varying partition counts.
In this version when a worker is subscribed to partitions:
[ TP(topic='foo', partition=0), TP(topic='foo', partition=1), TP(topic='foo', partition=2), TP(topic='foo', partition=3), TP(topic='bar', partition=0), TP(topic='bar', partition=1), TP(topic='bar', partition=2), TP(topic='bar', partition=3), TP(topic='baz', partition=0) ]
Note
TP
is short for topic and partition.When processing messages in these partitions, the worker will round robin between the topics in such a way that each topic will have an equal chance of being processed.
Transports: Fixed crash in aiokafka transport.
The worker would attempt to commit an empty set of partitions, causing an exception to be raised. This has now been fixed.
Stream: Removed unused method
Stream.tee
.This method was an example implementation and not used by any of our internal apps.
- Stream: Fixed bug when something raises
StopAsyncIteration
while processing the stream.
The Python async iterator protocol mandates that it’s illegal to raise
StopAsyncIteration
in an__aiter__
method.Before this change, code such as this:
async for value in stream: value = anext(other_async_iterator)
where
anext
raisesStopAsyncIteration
, Python would have the outer__aiter__
reraise that exception as:RuntimeError('__aiter__ raised StopAsyncIteration')
This no longer happens as we catch the
StopAsyncIteration
exception early to ensure it does not propagate.
- Stream: Fixed bug when something raises
1.0.1¶
- release-date
2018-05-01 9:52 A.M PDT
- release-by
Ask Solem
Stream: Fixed issue with using
break
when iterating over stream.The last message in a stream would not be acked if the
break
keyword was used:async for value in stream: if value == 3: break
Stream:
.take
now acks events after buffer processed.Previously the events were erroneously acked at the time of entering the buffer.
Note
To accomplish this we maintain a list of events to ack as soon as the buffer is processed. The operation is
O(n)
wheren
is the size of the buffer, so please keep buffer sizes small (e.g. 1000).A large buffer will increase the chance of consistency issues where events are processed more than once.
Stream: New
noack
modifier disables acking of messages in the stream.Use this to disable automatic acknowledgment of events:
async for value in stream.noack(): # manual acknowledgment await stream.ack(stream.current_event)
Manual Acknowledgment
The stream is a sequence of events, where each event has a sequence number: the “offset”.
To mark an event as processed, so that we do not process it again, the Kafka broker will keep track of the last committed offset for any topic.
This means “acknowledgment” works quite differently from other message brokers, such as RabbitMQ where you can selectively ack some messages, but not others.
If the messages in the topic look like this sequence:
1 2 3 4 5 6 7 8
You can commit the offset for #5, only after processing all events before it. This means you MUST ack offsets (1, 2, 3, 4) before being allowed to commit 5 as the new offset.
Stream: Fixed issue with
.take
not properly respecting thewithin
argument.The new implementation of take now starts a background thread to fill the buffer. This avoids having to restart iterating over the stream, which caused issues.
1.0.0¶
- release-date
2018-04-27 4:13 P.M PDT
- release-by
Ask Solem
Models: Raise error if
Record.asdict()
is overridden.Models: Can now override
Record._prepare_dict
to change the payload generated.For example if you want your model to serialize to a dictionary, but not have any fields with
None
values, you can override_prepare_dict
to accomplish this:class Quote(faust.Record): ask_price: float = None bid_price: float = None def _prepare_dict(self, data): # Remove keys with None values from payload. return {k: v for k, v in data.items() if v is not None} assert Quote(1.0, None).asdict() == {'ask_price': 1.0}
Stream: Removed annoying
Flight Recorder
logging that was too noisy.
Change history for Faust 0.9¶
This document contain historical change notes for bugfix releases in the Faust 0.x series. To see the most recent changelog please visit Change history for Faust 1.5.
0.9.65¶
- release-date
2018-04-27 2:04 P.M PDT
- release-by
Vineet Goel
Producer: New setting to configure compression.
Documentation: New Advanced Producer Settings section.
0.9.64¶
- release-date
2018-04-26 4:48 P.M PDT
- release-by
Ask Solem
Models: Optimization for
FieldDescriptor.__get__
.Serialization: Optimization for
faust.utils.json
.
0.9.63¶
- release-date
2018-04-26 04:32 P.M PDT
- release-by
Vineet Goel
Requirements:
Now depends on aiokafka 0.4.5 (Robinhood fork).
Models:
Record.asdict()
andto_representation()
were slow on complicated models, so we are now using code generation to optimize them.Warning
You are no longer allowed to override
Record.asdict()
.
0.9.62¶
- release-date
2018-04-26 12:06 P.M PDT
- release-by
Ask Solem
Requirements:
Now depends on Mode 1.12.2.
Now depends on aiokafka 0.4.4 (Robinhood fork).
Consumer: Fixed
asyncio.base_futures.IllegalStateError
error in commit handler.CLI: Fixed bug when invoking worker using
faust -A
.
Authors¶
Creators¶
Name |
|
---|---|
Ask Solem |
|
Vineet Goel |
Note
You must not solicit for free support from email addresses on this list. Ask the community for help in the Slack channel, or ask a question on Stack Overflow.
Committers¶
Arpan Shah |
|
Sanyam Satia |
Contributors become committers by stepping up to the task. They can 1) triage issues, help others on the issue tracker, code reviews, Slack or mailing lists, or 2) make modifications to documentation and code. The award for doing this in any significant capacity for one year or longer, is to be added to the list of maintainers above.
Contributors¶
Allison Wang |
|
Jamshed Vesuna |
|
Jaren Glover |
|
Jerry Li |
|
Prithvi Narasimhan |
|
Ruby Wang |
|
Shrey Kumar Shahi |
|
Mika Eloranta |
|
Omar Rayward |
|
Alexander Oberegger |
|
Matthew Stump |
|
Martin Maillard |
|
Mattias Karlsson |
|
Matthias Wutte |
|
Thibault Serot |
|
Ryan Whitten |
|
Nimi Wariboko Jr |
|
Chris Seto |
|
Amit Ripshtos |
|
Miha Troha |
|
Perk Lim |
|
Julien Surloppe |
|
Bob Haddleton |
Glossary¶
- acked
- acking
- acknowledged
Acknowledgment marks a message as fully processed. It’s a signal that the program does not want to see the message again. Faust advances the offset by committing after a message is acknowledged.
- agent
An async function that iterates over a stream. Since streams are infinite the agent will usually not end unless the program is shut down.
- codec
A codec encodes/decodes data to some format or encoding. Examples of codecs include Base64 encoding, JSON serialization, pickle serialization, text encoding conversion, and more.
- concurrent
- concurrency
A concurrent process can deal with many things at once, but not necessarily execute them in parallel. For example a web crawler may have to fetch thousands of web pages, and can work on them concurrently.
This is distinct from parallelism in that the process will switch between fetching web pages, but not actually process any of them at the same time.
- consumer
A process that receives messages from a broker, or a process that is actively reading from a topic/channel.
- event
A happening in a system, or in the case of a stream, a single record having a key/value pair, and a reference to the original message object.
- idempotence
- idempotent
- idempotency
Idempotence is a mathematical property that describes a function that can be called multiple times without changing the result. Practically it means that a function can be repeated many times without unintended effects, but not necessarily side-effect free in the pure sense (compare to nullipotent).
Further reading: https://en.wikipedia.org/wiki/Idempotent
- message
The unit of data published or received from the message transport. A message has a key and a value.
- nullipotent
- nullipotence
- nullipotency
describes a function that’ll have the same effect, and give the same result, even if called zero or multiple times (side-effect free). A stronger version of idempotent.
- parallel
- parallelism
A parallel process can execute many things at the same time, which will usually require running on multiple CPU cores.
In contrast the term concurrency refers to something that is seemingly parallel, but does not actually execute at the same time.
- publisher
A process sending messages, or a process publishing data to a topic.
- reentrant
- reentrancy
describes a function that can be interrupted in the middle of execution (e.g., by hardware interrupt or signal), and then safely called again later. Reentrancy isn’t the same as idempotence as the return value doesn’t have to be the same given the same inputs, and a reentrant function may have side effects as long as it can be interrupted; An idempotent function is always reentrant, but the reverse may not be true.
- sensor
A sensor records information about events happening in a running Faust application.
- serializer
A serializer is a codec, responsible for serializing keys and values in messages sent over the network.
- task
A task is the unit of concurrency in an
asyncio
program.- thread safe
A function or process that is thread safe means multiple POSIX threads can execute it in parallel without race conditions or deadlock situations.
- topic
Consumers subscribe to topics of interest, and producers send messages to consumers via the topic.
- transport
A communication mechanism used to send and receive messages, for example Kafka.