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 superfast 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 Quickstart tutorial
to see Faust in action by programming a streaming application.
then explore the User Guide
for in-depth information organized by topic.

Contents

Introducing Faust

Version:1.0.30
Web:http://faust.readthedocs.io/
Download:http://pypi.org/project/faust
Source:http://github.com/robinhood/faust
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 new async/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 existing asyncio/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, Scikit, TensorFlow, etc.

Faust is used for…

  • Event Processing
  • Distributed Joins & Aggregations
  • Machine Learning
  • Asynchronous Tasks
  • Distributed Computing
  • Data Denormalization
  • Intrusion Detection
  • Realtime Web & Web Sockets.
  • and much more…

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
rocksdb 5.0 pip install faust[rocksdb]
statsd 3.2.1 pip install faust[statsd]
uvloop 0.8.1 pip install faust[uvloop]
aiodns 1.1 pip install faust[fast]
setproctitle 1.1 pip install faust[setproctitle] (also debug)
aiomonitor 0.3 pip install faust[debug]

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,statsd]

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/

License

This software is licensed under the New BSD License. See the LICENSE file in the top distribution directory for the full license text.

Playbooks

Release:1.0
Date:Aug 17, 2018

Quickstart

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 Quickstart 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 Quickstart 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 Quickstart 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

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 Quickstart 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 as map, but ends the processing chain.

  • .peek()

    In KS peek is the same as map, 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 asyncio
    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 transfer for transfers 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 Introduction to Mode introduction page
to read more about Faust, system requirements, installation instructions, community resources, and more.
or go directly to the Quickstart tutorial
to see Faust in action by programming a streaming application.
then explore the User Guide
for in-depth information organized by topic.

Cheatsheet

Process events in a Kafka topic

import faust

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.0
Date:Aug 17, 2018

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:// RocksDB an embedded database (production)
memory:// 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:
        ...
Topic Arguments
  • key_type/value_type: ModelArg

    Use the key_type and value_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 is bytes and treats the key as a binary string. The key can also be specified as a model type (key_type=MyKeyModel).

    See also

  • 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 and value_serializer settings.

    See also

  • 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

    Number of seconds (as float/timedelta) to keep messages in the topic before they can be expired by the server.

  • 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 acknowledgement for this topic. If you disable this then you are responsible for manually acknowleding 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

Channel Arguments
  • key_type/value_type: ModelArg

    Use the key_type and value_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 channell

    If not specified the default will be taken from the key_serializer and value_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”.

Table Arguments
  • 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 to 0 and return 0:

    >>> table = app.Table('hasdefault', default=int)
    
    >>> table['missing']
    0
    
  • key_type/value_type: ModelArg

    Use the key_type and value_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)
    
  • store: str or URL

    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.Set() – Define a new Set-based table

Use Set() to create a set table that only tracks membership and does not associate each key with a particular value:

users_with_transfer = app.Set('users-with-transfers', key_type=str)

@app.agent(transfers_topic)
async def transfer(transfers):
    async for transfer in transfers:
        users_with_transfer.add(transfer.username)

See also

Set Arguments

Supports the same arguments as table(): see Table Arguments.

@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

Agent Arguments
  • 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 snould 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:

    • mode.OneForAllSupervisor

      If one agent instance of this type raises an exception we will restart all other agent instances of this type

    • mode.CrashingSupervisor

      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

@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

Timer Arguments
  • on_leader: bool

    If enabled this timer will only execute on one of the worker instances at a time – that is only on the leader of the cluster.

    This can be used as a distributed mutex to execute something on one machine at a time.

@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

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
impor 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

@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
sender:faust.App
arguments:Set[TP]

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_assigned(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
sender:faust.App
arguments:Set[TP]

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:faust.App
arguments:faust.Settings
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_url = os.environ.get('FAUST_BROKER')
    conf.store_url = os.environ.get('STORE_URL')
App.on_before_configured
sender:faust.App
arguments:none
synchronous:This is a synchronous signal (do not use async def).

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
def before_configuration(app, **kwargs):
    print(f'App {app} is being configured')
App.on_after_configured
sender:faust.App
arguments:none
synchronous:This is a synchronous signal (do not use async def).

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
def after_configuration(app, **kwargs):
    print(f'App {app} has been configured.')
App.on_worker_init
sender:faust.App
arguments:none
synchronous:This is a synchronous signal (do not use async def).

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
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 accomodate the many ways you may want to embed a Faust application, starting the app have several possible entrypoints:

App entrypoints:

  1. 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 entrypoints to setup.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 using from 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
    
  2. -> 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 entrypoint 3 or 4).
    • Starting the faust.Worker (see next step).
  3. -> 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 entrypoint 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 backdoor.
      • Starting the blocking detector.
    • Setting up TERM and INT 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,

    1. from synchronous code, use Worker.execute_from_commandline:

      >>> worker = Worker(app)
      >>> worker.execute_from_commandline()
      
    2. or from an async def function call await 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).

  4. -> 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 configured logging manually, and it won’t set up any TERM/INT signal handlers, which means finally blocks won’t execute at shutdown.

    Start app directly:

    async def start_app(app):
        await app.start()
    

    Web server

    Starting the app will not start the webserver, as this is part of the workers responsibility.

    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 depdendency 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()
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.

Reference

See faust.App in the API reference for a full list of methods and attributes supported.

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 the async 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 tolerancy

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 prefes 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 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

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 aren’t allowed to 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')

@app.agent()
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 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):
    ...
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):
    ...
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 event in stream:
        print(f'AGENT B RECEIVED: {event!r}')

@app.agent(sink=[agent_b])
async def agent_a(stream):
    async for event in stream:
        print(f'AGENT A RECEIVED: {event!r}')
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 unhandled 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 by cast and ask.

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 orering 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 use use join 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: First part

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 argumetns.

See also

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 iterater probably 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 Lifecycle
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.

Acknowledgment

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() – Repartiton 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

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 metadata:

@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 in event.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.aenumerate() 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 aenumerate also accepts an optional starting point argument.

See also

  • The faust.utils.aiter.aenumerate() function – for a general version of enumerate() 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

Reference

Note

Do not create Stream objects directly, instead use: app.stream to instantiate new streams.

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
Sending messages to channel
class faust.Channel[source]
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send message to channel.

Return type:Awaitable[RecordMetadata]
as_future_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → faust.types.tuples.FutureMessage[source]
Return type:FutureMessage[]
coroutine publish_message(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
Declaring

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).

class faust.Channel[source]
coroutine maybe_declare()[source]
coroutine declare(self) → None[source]
Return type:None
Topics

A topic is a named channel, backed by a Kafka topic. The name is used as the address of the channel, to share it between multiple processes and each process will receive a partition of the topic.

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)
Lists of lists, etc.

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
  • Tuple[ModelT, ...]
  • Tuple[ModelT, ModelT, str]
Mapping

From this table we can see that we can also have a mapping of username to account:

from typing import Mapping
import faust

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.

Automatic conversion of datetimes

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
Subclassing models: Abstract model classes

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)
Positional Arguments

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)
Reference
Serialization/Deserialization
class faust.Record[source]
classmethod loads(s: bytes, *, default_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → faust.types.models.ModelT

Deserialize model object from bytes.

Parameters:
  • default_serializer (CodecArg) – Default serializer to use if no custom serializer was set for this model subclass.
  • **kwargs – Additional attributes to set on the model object. Note, these are regarded as defaults, and any fields also present in the message takes precedence.
Return type:

ModelT

dumps(*, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → bytes

Serialize object to the target serialization format.

Return type:bytes
to_representation() → Mapping[str, Any][source]

Convert object to JSON serializable object.

Return type:Mapping[str, Any]
classmethod from_data(data: Mapping, *, preferred_type: Type[faust.types.models.ModelT] = None) → faust.models.record.Record[source]
Return type:Record
derive(*objects, **fields) → faust.types.models.ModelT
Return type:ModelT
_options

Model metadata for introspection. An instance of faust.types.models.ModelOptions.

class faust.ModelOptions[source]
fields = None

Index – Flattened view of __annotations__ in MRO order.

fieldset = None

Index – Set of required field names, for fast argument checking.

fieldpos = None

Index – Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.

optionalset = None

Index – Set of optional field names, for fast argument checking.

models = None

Index – Mapping of fields that are ModelT

converse = None

Index – Mapping of fields that are not builtin-types. E.g. datetime.

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, so we are hoping to take advantage of Kafka 0.11’s stronger consistency guarantees and new “exactly-once”-semantics features as soon as that is supported in a Python Kafka client.

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.

Repartion 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 reimplement 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

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 HoppingWindow
class TumblingWindow
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:

events_topic = app.topic('events_elk', value_type=Event)

@app.agent(events_topic)
async def aggregate_page_views(events):
    async for event in events:
        page = event.page

        # increment one to all windows this event falls into.
        views[page] += 1

        if views[page].now() >= 10000:
            # Page is trending for current processing time window
            print('Trending now')
        if views[page].current(event) >= 10000:
            # Page would be trending in the event's time window
            print('Trending when event happened')
        if views[page].delta(timedelta(minutes=30)) > views[page].now():
            print('Less popular compared to 30 minutes back')

In this table, table[k].current() 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].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
        print(table[key].delta(30))
“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, 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.

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 apps 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:

… sourcecode:: python

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/')
@app.table_route(table=word_counts, shard_param='word')
async def get_count(web, request):
    word = request.GET['word']
    return web.json({
        word.word: word_counts[word.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’ve added a produce command used to send example data into the stream processors:

from click 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.

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 entrypoint 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 entrypoints so that pip install myproj installs a command-line program.

Even if you don’t add an entrypoint 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:

  1. 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).

--web-host, -h

Canonical host name for the web server.

--web-port, -p

Port to run web server on.

--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,271: INFO]: [^-Website]: 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]: [^-App]: Starting...
[2018-03-13 13:41:39,275: INFO]: [^--MonitorService]: 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 version of the monitor available that forwards statistics to StatsD.

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
Class: Monitor
Monitor Attributes
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

List of run times used for averages

events_runtime_avg

Average event runtime over the last second.

tables

Mapping of tables

commit_latency

List of commit latency values

send_latency

List of send latency values

messages_sent

Number of messages sent in total.

messages_sent_by_topic

Number of messages sent by topic.

Configuration Attributes
class faust.Monitor[source]
max_avg_history = 0

Max number of total run time values to keep to build average.

max_commit_latency_history = 0

Max number of commit latency numbers to keep.

max_send_latency_history = 0

Max number of send latency numbers to keep.

Class: 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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → 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():
    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:"aiokafka://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:

aiokafka://kafka1.example.com:9092;kafka2.example.com:9092
store
type:str
default: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.

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 the origin attribute.

The origin attribute is automatically set when you start a worker using the faust command line program, for example:

.. sourcecode:: console
faust -A example.simple worker

The -A, option specifies the app, but you can also create a shortcut entrypoint entrypoint by calling app.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 apps. 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:int
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).

datadir
type:Union[str, pathlib.Path]
default:"{appid}-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:str
default:"{id}-v{self.version}"

The format string used to generate the final id value by combining it with the version parameter.

loghandlers
type:List[logging.LogHandler]
default:None

Specify a list of custom log handlers to use in worker instances.

origin
type:str
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 autogenerated names then you can set origin manually.

Serialization Settings
key_serializer
type:Union[str, Codec]
default:"json"

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:int
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:int
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).

Advanced Broker Settings
broker_client_id
type:str
default:faust-{VERSION}

You shouldn’t have to set this manually.

The client id is used to identify the software used, and is not usually configured by the user.

broker_commit_every
type:int
default:1000

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_interval
type:float, timedelta
default:2.8

How often we commit messages that have been fully processed (acked).

broker_commit_livelock_soft_timeout
type:class:float, timedelta
default:300.0 (five minutes)

How long time it takes before we warn that the Kafka commit offset has not advanced (only when processing messages).

broker_check_crcs
type:bool
default:True

Automatically check the CRC32 of the records consumed.

broker_heartbeat_interval
type:int
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
type:int
default:30.0 (thirty seconds)

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.

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:int
default:0

Minimum time to batch before sending out messages from the producer.

Should rarely have to change this.

producer_max_batch_size
type:int
default:16384

Max number of records in each producer batch.

producer_max_request_size
type:int
default:1000000

Maximum size of a request in bytes in the producer.

Should rarely have to change this.

producer_acks
type:int
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 acknowledgement 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.
Advanced Table Settings
table_cleanup_interval
type:float, timedelta
default:30.0

How often we cleanup tables to remove expired entries.

table_standby_replicas
type:int
default:1

The number of standby replicas for each table.

Advanced Stream Settings
stream_buffer_maxsize
type:int
default:4096

This setting control backpressure 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), Faust 1.0 will move the sending of those messages to when the offset of the source message (the one that initiated the sending/change) is committed.

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 backpressure 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.

The next version of Faust will take advantage of Kafka transactions to remove the bottleneck of sending messages on commit.

stream_wait_empty
type:bool
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_ack_exceptions
type:bool
default:True

What happens when an exception is raised while processing an event? We ack that message by default, so we never reprocess it. This may be surprising, but it avoids the very likely scenario of causing a high frequency loop, where the error constantly happens and we never recover.

You can set this to False to reprocess events that caused an exception to be raised.

stream_ack_cancelled_tasks
type:bool
default:False

What happens when processing an event and the task processing it is cancelled? Agent tasks can be cancelled during shutdown or rebalance, and by default we do not ack the task in this case, so the event can be reprocessed.

If your agent processors are not idempotent you may want to set this flag to True, so that once processing an event started, it will not process that event again.

stream_publish_on_commit
type:bool
default:False

We buffer up sending messages until the source topic offset related to that processsing is committed. This means when we do commit, we may have buffered up a LOT of messages so commit frequently.

This setting will be removed once transaction support is added in a later version.

Advanced Worker Settings
worker_redirect_stdouts
type:bool
default:True

Enable to have the worker redirect output to sys.stdout and sys.stderr to the Python logging system.

Enabled by default.

worker_redirect_stdouts_level
type:str/int
default::data:logging.WARN

The logging level to use when redirect STDOUT/STDERR to logging.

Advanced Web Server Settings
canonical_url
type:str
default:socket.gethostname()

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.

Advanced Agent Settings
agent_supervisor
type:str:/mode.SupervisorStrategyT
default:mode.OneForOneSupervisor

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:

Agent RPC Settings
reply_to
type:str
default:<generated>

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.

reply_expires
type:Union[float, datetime.timedelta]
default:timedelta(days=1)

The expiry time (in seconds float, or timedelta), for how long replies will stay in the instances local reply topic before being removed.

reply_to_prefix
type:str
default:"f-reply-"

The prefix used when generating reply topic names.

Extension Settings
Agent
type:Union[str, Type]
default:"faust.Agent"

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:"faust.Stream"

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:"faust.Table"

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')
TableManager
type:Union[str, Type[TableManagerT]]
default:"faust.tables.TableManager"

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:"faust.Worker"

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:"faust.app.router.Router"

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:"faust.Topic"

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:"faust.sensors:Monitor"

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]"

The following bundles are available:

Stores
faust[rocksdb]:

for using RocksDB for storing Faust table state.

Recommended in production.

Optimization
faust[fast]:for installing all the available C speedup extensions to Faust core.
Sensors
faust[statsd]:for using the Statsd Faust monitor.
Event Loops
faust[uvloop]:for using Faust with uvloop.
faust[gevent]:for using Faust with gevent.
faust[eventlet]:
 for using Faust with eventlet
Debugging
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 and debug 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.

Using the development version
With pip

You can install the latest snapshot of Faust using the following pip command:

$ pip install https://github.com/robinhood/faust/zipball/master#egg=faust

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 methodology Kafka uses to make sure that as data for a key changes it doesn’t 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-01-04 12:41:07,638: INFO]: [^-Website]: 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 worker -l info --web-port=6066

Then start the second worker:

$ faust --datadir=/var/faust/worker2 -A proj worker -l info --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 processsing 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:

TERM Warm shutdown, wait for tasks to complete.
QUIT Cold shutdown, terminate ASAP
USR1 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/

API Reference

Release:1.0
Date:Aug 17, 2018

Faust

faust

Python Stream processing.

class faust.Agent(fun: Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, on_error: Callable[[_ForwardRef('AgentT'), BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, isolated_partitions: bool = False, **kwargs) → None[source]

Agent.

This is the type of object returned by the @app.agent decorator.

cancel() → None[source]
Return type:None
info() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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, **kwargs) → faust.types.agents.AgentTestWrapperT[source]
Return type:AgentTestWrapperT[]
actor_from_stream(stream: faust.types.streams.StreamT) → faust.types.agents.ActorT[Union[typing.AsyncIterable, typing.Awaitable]][source]
Return type:ActorT[]
add_sink(sink: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]) → None[source]
Return type:None
stream(active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
get_topic_names() → Iterable[str][source]
Return type:Iterable[str]
coroutine ask(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]
Return type:Any
coroutine ask_nowait(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]
Return type:ReplyPromise
coroutine cast(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None) → None[source]
Return type:None
channel
coroutine join(self, values: Union[typing.AsyncIterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]], typing.Iterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvjoin(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
kvmap(items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]
Return type:AsyncIterator[str]
logger = <Logger faust.agents.agent (WARNING)>
map(values: Union[typing.AsyncIterable, typing.Iterable], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('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
coroutine on_shared_partitions_assigned(self, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_shared_partitions_revoked(self, revoked: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, *, reply_to: Union[_ForwardRef('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:Awaitable[RecordMetadata]
channel_iterator
label
class faust.App(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = 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.

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).

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)
Parameters:
  • silent (bool) – If true then import errors will be ignored.
  • force (bool) – Force reading configuration immediately. By default the configuration will be read only when required.
Return type:

None

finalize() → None[source]
Return type:None
worker_init() → None[source]
Return type:None
discover(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[str] = ['test_.*', '.*__main__.*']) → None[source]
Return type:None
main() → None[source]

Execute the faust umbrella command using this app.

Return type:None
topic(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, 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().

Return type:TopicT[]
channel(*, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, maxsize: int = 1, 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).

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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
task(fun: Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.Awaitable]]) → Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.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')
Return type:Union[Callable[[AppT[]], Awaitable[+T_co]], Callable[[], Awaitable[+T_co]]]
timer(interval: Union[datetime.timedelta, float, str], on_leader: bool = False) → 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:Callable
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):
    ...
Return type:Type[ServiceT[]]
is_leader() → bool[source]
Return type:bool
stream(channel: Union[typing.AsyncIterable, typing.Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]

Create new stream from channel/topic/iterable/async iterable.

Parameters:
Return type:

StreamT[+T_co]

Returns:

to iterate over events in the stream.

Return type:

faust.Stream

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[]
page(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>) → Callable[Union[typing.Type[faust.types.web.View], typing.Callable[[faust.types.web.View, faust.types.web.Request], typing.Awaitable[faust.types.web.Response]]], Type[faust.web.views.Site]][source]
Return type:Callable[[Union[Type[View], Callable[[View, Request], Awaitable[Response]]]], Type[Site]]
table_route(table: faust.types.tables.CollectionT, shard_param: str) → Callable[Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]], Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]]][source]
Return type:Callable[[Callable[[View, Request], Awaitable[Response]]], Callable[[View, Request], Awaitable[Response]]]
command(*options, base: Union[typing.Type[faust.app.base.AppCommand], NoneType] = None, **kwargs) → Callable[Callable, Type[faust.app.base.AppCommand]][source]
Return type:Callable[[Callable], Type[AppCommand]]
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:ThrowableQueue
Worker(**kwargs) → faust.app.base.WorkerT[source]
Return type:WorkerT
on_webserver_init(web: faust.web.base.Web) → None[source]
Return type:None
conf
producer[source]
consumer
transport

Message transport. :rtype: TransportT

tables[source]

Map of available tables, and the table manager service.

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:bool
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
maybe_start_producer[source]

Ensure producer is started.

coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine send(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send event to channel/topic.

Parameters:
Return type:

Awaitable[RecordMetadata]

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
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.

monitor

Monitor keeps stats about what’s going on inside the worker. :rtype: Monitor[]

flow_control[source]

Internal flow control.

This object controls flow into stream queues, and can also clear all buffers.

http_client

HTTP Client Session. :rtype: HttpClientT

assignor[source]

Partition Assignor.

Responsible for partition assignment.

router[source]

Find the node partitioned data belongs to.

The router helps us route web requests to the wanted Faust node. If a topic is sharded by account_id, the router can send us to the Faust worker responsible for any account. Used by the @app.table_route decorator.

serializers[source]
label
shortlabel
class faust.Channel(app: faust.types.app.AppT, *, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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 new put call will block until a message is removed from the channel.
  • loop (Optional[AbstractEventLoop]) – The asyncio event loop to use.
coroutine deliver(self, message: faust.types.tuples.Message) → None[source]
Return type:None
queue
clone(*, is_iterator: bool = None, **kwargs) → 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]
get_topic_name() → str[source]
Return type:str
as_future_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → faust.types.tuples.FutureMessage[source]
Return type:FutureMessage[]
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
empty() → bool[source]
Return type:bool
coroutine declare(self) → None[source]
Return type:None
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:Any
maybe_declare[source]
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
on_stop_iteration() → 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:Awaitable[RecordMetadata]
coroutine put(self, value: Any) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send message to channel.

Return type:Awaitable[RecordMetadata]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
subscriber_count
label
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: Union[faust.types.channels.ChannelT, NoneType] = 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[]
stream(**kwargs) → faust.types.channels.StreamT[source]
Return type:StreamT
get_topic_name() → str[source]
Return type:str
as_future_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → faust.types.tuples.FutureMessage[source]
Return type:FutureMessage[]
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
empty() → bool[source]
Return type:bool
on_stop_iteration() → None[source]
Return type:None
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
subscriber_count
queue
coroutine declare(self) → None[source]
Return type:None
coroutine decode(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.channels.EventT[source]
Return type:EventT
coroutine deliver(self, message: faust.types.tuples.Message) → None[source]
Return type:None
coroutine get(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]
Return type:Any
maybe_declare[source]
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:Awaitable[RecordMetadata]
coroutine put(self, value: Any) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
class faust.Event(app: faust.types.app.AppT, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], 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

  • Iteratiing 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, typing.Any, NoneType] = <object object>, value: Union[bytes, faust.types.core.ModelT, typing.Any] = <object object>, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Forward original message (will not be reserialized).

Return type:Awaitable[RecordMetadata]
coroutine send(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = <object object>, value: Union[bytes, faust.types.core.ModelT, typing.Any] = <object object>, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send object to channel.

Return type:Awaitable[RecordMetadata]
ack() → bool[source]
Return type:bool
class faust.EventT(app: faust.types.events.AppT, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], message: faust.types.tuples.Message) → None[source]
app
key
value
message
acked
ack() → bool[source]
Return type:bool
coroutine forward(self, channel: Union[str, faust.types.events.ChannelT], key: Any = None, value: Any = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send(self, channel: Union[str, faust.types.events.ChannelT], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
class faust.ModelOptions(*args, **kwargs)[source]
serializer = None
include_metadata = True
allow_blessed_key = False
isodates = False
clone_defaults() → faust.types.models.ModelOptions[source]
Return type:ModelOptions
fields = None

Index – Flattened view of __annotations__ in MRO order.

fieldset = None

Index – Set of required field names, for fast argument checking.

fieldpos = None

Index – Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.

optionalset = None

Index – Set of optional field names, for fast argument checking.

models = None

Index – Mapping of fields that are ModelT

modelattrs = None
converse = None

Index – Mapping of fields that are not builtin-types. E.g. datetime.

defaults = None

Mapping of field names to default value.

initfield = None

Mapping of init field conversion callbacks.

class faust.Record(*args, __strict__: bool = True, _Record__faust: Any = None, **kwargs) → 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:Record
to_representation() → Mapping[str, Any][source]

Convert object to JSON serializable object.

Return type:Mapping[str, Any]
asdict() → Dict[str, Any][source]
Return type:Dict[str, Any]
class faust.Monitor(*, max_avg_history: int = 100, max_commit_latency_history: int = 30, max_send_latency_history: int = 30, 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: List[float] = None, commit_latency: List[float] = None, send_latency: List[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, **kwargs) → None[source]

Default Faust Sensor.

This is the default sensor, recording statistics about events, etc.

max_avg_history = 0

Max number of total run time values to keep to build average.

max_commit_latency_history = 0

Max number of commit latency numbers to keep.

max_send_latency_history = 0

Max number of send latency numbers to keep.

tables = None

Mapping of tables

commit_latency = None

List of commit latency values

send_latency = None

List of send latency values

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

List 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

asdict() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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_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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

Return type:None
count(metric_name: str, count: int = 1) → None[source]
Return type:None
on_tp_commit(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]
Return type:None
track_tp_end_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

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.

kwargs = None

subclasses can support keyword arguments, the base implementation of clone() uses this to preserve keyword arguments in copies.

dumps(obj: Any) → bytes[source]

Encode object obj.

Return type:bytes
loads(s: bytes) → Any[source]

Decode object from string.

Return type:Any
clone(*children) → faust.types.codecs.CodecT[source]

Create a clone of this codec, with optional children added.

Return type:CodecT
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, loop: asyncio.events.AbstractEventLoop = None) → None[source]

A stream: async iterator processing events in channels/topics.

logger = <Logger faust.streams (WARNING)>
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 be c.

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]
get_root_stream() → faust.types.streams.StreamT[source]
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
info() → Mapping[str, Any][source]

Return stream settings as a dictionary.

Return type:Mapping[str, Any]
clone(**kwargs) → faust.types.streams.StreamT[source]

Create a clone of this stream.

Notes

If the cloned stream is supposed to “supercede” this stream, like in group_by/through/etc., you should use _chain() instead so stream._next = cloned_stream is set and get_active_stream() returns the cloned stream.

Return type:StreamT[+T_co]
noack() → faust.types.streams.StreamT[source]
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:AsyncIterable[EventT[]]
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, typing.Callable[[~T], typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType]]], *, 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.
  • 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[]

combine(*nodes, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
contribute_to_stream(active: faust.types.streams.StreamT) → None[source]
Return type:None
join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
left_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
inner_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
outer_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
coroutine on_merge(self, value: T = None) → Union[~T, NoneType][source]
Return type:Optional[~T]
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.

Parameters:event (EventT[]) – Event to ack.
Return type:bool
items() → AsyncIterator[Tuple[Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], 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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
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
coroutine stop(self) → None[source]

Stop the service.

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]]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel[source]
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: Union[faust.types.streams.StreamT, NoneType] = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, 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
get_active_stream() → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
add_processor(processor: Callable[T]) → None[source]
Return type:None
info() → Mapping[str, Any][source]
Return type:Mapping[str, Any]
clone(**kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
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]
echo(*channels) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
group_by(key: Union[faust.types.models.FieldDescriptorT, typing.Callable[[~T], typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType]]], *, 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]
Return type:TopicT[]
coroutine ack(self, event: faust.types.events.EventT) → bool[source]
Return type:bool
coroutine events(self) → AsyncIterable[faust.types.events.EventT][source]
coroutine items(self) → AsyncIterator[Tuple[Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], T_co]][source]
coroutine send(self, value: T_contra) → None[source]
Return type:None
coroutine take(self, max_: int, within: Union[datetime.timedelta, float, str]) → AsyncIterable[Sequence[T_co]][source]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
faust.current_event() → Union[faust.types.events.EventT, NoneType][source]

Return the event currently being processed, or None.

Return type:Optional[EventT[]]
class faust.Table(app: faust.types.app.AppT, *, name: str = None, default: Callable[Any] = None, store: Union[str, yarl.URL] = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]

Table (non-windowed).

using_window(window: faust.types.windows.WindowT) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
hopping(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
tumbling(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
on_key_get(key: Any) → None[source]

Handle that key is being retrieved.

Return type:None
on_key_set(key: Any, value: Any) → None[source]

Handle that value for a key is being set.

Return type:None
on_key_del(key: Any) → None[source]

Handle that a key is deleted.

Return type:None
as_ansitable(*, key: str = 'Key', value: str = 'Value', sort: bool = False, sortkey: Callable[Any, Any] = operator.itemgetter(0), target: IO = <_io.TextIOWrapper name='<stdout>' mode='w' encoding='UTF-8'>, title: str = '{table.name}') → str[source]
Return type:str
logger = <Logger faust.tables.table (WARNING)>
class faust.Topic(app: faust.types.app.AppT, *, topics: Sequence[str] = None, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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]

Define new topic description.

Parameters:
  • app (AppT[]) – App instance used to create this topic description.
  • topics (Optional[Sequence[str]]) – List of topic names.
  • 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, and autoCreateTopics 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 a faust.Model type, str, bytes, or None for “autodetect”
  • value_type (Union[Type[ModelT], Type[bytes], Type[str], None]) – How to deserialize values for messages in this topic. Can be a faust.Model type, str, bytes, or None for “autodetect”
  • active_partitions (Optional[Set[TP]]) – Set of faust.types.tuples.TP that this topic should be restricted to.
Raises:

TypeError – if both topics and pattern is provided.

pattern
derive(**kwargs) → faust.types.channels.ChannelT[source]

Create new Topic derived from this topic.

Configuration will be copied from this topic, but any parameter overriden 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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[]
get_topic_name() → str[source]
Return type:str
coroutine declare(self) → None[source]
Return type:None
coroutine decode(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]
Return type:EventT[]
maybe_declare[source]
coroutine publish_message(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine put(self, event: faust.types.events.EventT) → None[source]
Return type:None
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
on_stop_iteration() → None[source]
Return type:None
partitions
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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]
topics = None

Iterable/Sequence of topic names to subscribe to.

retention = None

Topic retention setting – expiry time in seconds for messages in the topic.

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

Mark topic as internal – it’s owned by us and we are allowed to create or delete the topic as necessary.

pattern

or instead of topics, a regular expression used to match topics we want to subscribe to. :rtype: Optional[Pattern[AnyStr]]

partitions
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
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.Settings(id: str, *, version: int = None, broker: Union[str, yarl.URL] = None, broker_client_id: str = 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, agent_supervisor: Union[typing.Type[mode.types.supervisors.SupervisorStrategyT], str] = None, store: Union[str, yarl.URL] = None, autodiscover: Union[bool, typing.Iterable[str], typing.Callable[[], typing.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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, loghandlers: List[logging.StreamHandler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = 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, 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, 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, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[typing.Type[faust.types.agents.AgentT], str] = None, Stream: Union[typing.Type[faust.types.streams.StreamT], str] = None, Table: Union[typing.Type[faust.types.tables.TableT], str] = None, TableManager: Union[typing.Type[faust.types.tables.TableManagerT], str] = None, Serializers: Union[typing.Type[faust.types.serializers.RegistryT], str] = None, Worker: Union[typing.Type[faust.types.settings.WorkerT], str] = None, PartitionAssignor: Union[typing.Type[faust.types.assignor.PartitionAssignorT], str] = None, LeaderAssignor: Union[typing.Type[faust.types.assignor.LeaderAssignorT], str] = None, Router: Union[typing.Type[faust.types.router.RouterT], str] = None, Topic: Union[typing.Type[faust.types.topics.TopicT], str] = None, HttpClient: Union[typing.Type[faust.types.web.HttpClientT], str] = None, Monitor: Union[typing.Type[faust.types.sensors.SensorT], str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None[source]
classmethod setting_names() → Set[str][source]
Return type:Set[str]
id_format = '{id}-v{self.version}'
origin = None
autodiscover = False
broker_client_id = 'faust-1.0.30'
broker_commit_every = 10000
broker_check_crcs = True
key_serializer = 'json'
value_serializer = 'json'
table_standby_replicas = 1
topic_replication_factor = 1
topic_partitions = 8
reply_create_topic = False
stream_buffer_maxsize = 4096
stream_wait_empty = False
stream_ack_cancelled_tasks = False
stream_ack_exceptions = True
stream_publish_on_commit = True
producer_linger_ms = 0
producer_max_batch_size = 16384
producer_acks = -1
producer_max_request_size = 1000000
producer_compression_type = None
worker_redirect_stdouts = True
worker_redirect_stdouts_level = 'WARN'
reply_to_prefix = 'f-reply-'
prepare_id(id: str) → str[source]
Return type:str
prepare_datadir(datadir: Union[str, pathlib.Path]) → pathlib.Path[source]
Return type:Path
prepare_tabledir(tabledir: Union[str, pathlib.Path]) → pathlib.Path[source]
Return type:Path
name
id
version
broker
store
canonical_url
datadir
appdir
tabledir
broker_session_timeout
broker_heartbeat_interval
broker_commit_interval
broker_commit_livelock_soft_timeout
table_cleanup_interval
reply_expires
agent_supervisor
Agent
Stream
Table
TableManager
Serializers
Worker
PartitionAssignor
LeaderAssignor
Router
Topic
HttpClient
Monitor
class faust.HoppingWindow(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → None[source]

Hopping window type.

Fixed-size, overlapping windows.

ranges(timestamp: float) → List[faust.types.windows.WindowRange][source]
Return type:List[WindowRange]
stale(timestamp: float, latest_timestamp: float) → bool[source]
Return type:bool
current(timestamp: float) → faust.types.windows.WindowRange[source]
Return type:WindowRange
delta(timestamp: float, d: Union[datetime.timedelta, float, str]) → faust.types.windows.WindowRange[source]
Return type:WindowRange
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.

class faust.SlidingWindow(before: Union[datetime.timedelta, float, str], after: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str]) → None[source]

Sliding window type.

Fixed-size, overlapping windows that work on differences between record timestamps

ranges(timestamp: float) → List[faust.types.windows.WindowRange][source]

Return list of windows from timestamp.

Notes

SELECT * FROM s1, s2
WHERE
    s1.key = s2.key
AND
s1.ts - before <= s2.ts AND s2.ts <= s1.ts + after
Return type:List[WindowRange]
stale(timestamp: float, latest_timestamp: float) → bool[source]
Return type:bool
class faust.Window(*args, **kwargs)[source]
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, typing.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, Website: Union[typing.Type[faust.web.site.Website], str] = 'faust.web.site:Website', web_port: int = None, web_bind: str = None, web_host: str = None, console_port: int = 50101, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]

Worker.

Usage:

You can start a worker using:

  1. the faust worker program.

  2. instantiating Worker programmatically and calling execute_from_commandline():

    >>> worker = Worker(app)
    >>> worker.execute_from_commandline()
    
  3. 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.
  • Website (Union[Type[Website[]]]) – Class used to serve the Faust web site (defaults to faust.web.site.Website).
  • web_port (int) – Port for web site to bind to (defaults to 6066).
  • web_bind (str) – Host to bind web site to (defaults to “0.0.0.0”).
  • web_host (str) – Canonical host name used for this server. (defaults to the current host name).
  • 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.

Website = None

Class that starts our web server and serves the Faust website.

web_port = None

Port to run the embedded web server on (defaults to 6066).

web_bind = None

Host to bind web server port to (defaults to ‘0.0.0.0’).

spinner = None

Class that displays a terminal progress spinner (see progress).

on_init_dependencies() → Iterable[mode.types.services.ServiceT][source]

Callback to be used to add service dependencies.

Return type:Iterable[ServiceT[]]
change_workdir(path: pathlib.Path) → None[source]
Return type:None
autodiscover() → None[source]
Return type:None
coroutine on_execute(self) → None[source]
Return type:None
coroutine on_first_start(self) → None[source]

Called only the first time the service is started.

Return type:None
coroutine on_startup_finished(self) → None[source]
Return type:None
on_setup_root_logger(logger: logging.Logger, level: int) → None[source]
Return type:None
website[source]
faust.uuid() → str[source]
Return type:str
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.

Notes

Instantiating a service will create the asyncio event loop. If your object is created as a side effect of importing a module, then you should use mode.proxy.ServiceProxy.

Keyword Arguments:
 
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.

classmethod task(fun: Callable[Any, Awaitable[NoneType]]) → mode.services.ServiceTask[source]

Decorator used to define a service 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[NoneType]], mode.services.ServiceTask][source]

A background timer that executes every n seconds.

Example

>>> class S(Service):
...
...     @Service.timer(1.0)
...     async def background_timer(self):
...         print('Waking up')
Return type:Callable[[Callable[[ServiceT[]], Awaitable[None]]], ServiceTask]
classmethod transitions_to(flag: str) → Callable[source]

Decorator that adds diagnostic flag while function is running.

Return type:Callable
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_context(context: ContextManager) → Any[source]
Return type:Any
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() → None[source]
Return type:None
on_init_dependencies() → Iterable[mode.types.services.ServiceT][source]

Callback to be used to add service dependencies.

Return type:Iterable[ServiceT[]]
coroutine add_async_context(self, context: AsyncContextManager) → Any[source]
Return type:Any
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
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 restart(self) → None[source]

Restart this service.

Return type:None
coroutine sleep(self, n: Union[datetime.timedelta, float, str]) → None[source]

Sleep for n seconds, or until service stopped.

Return type:None
coroutine start(self) → None[source]
Return type:None
coroutine stop(self) → None[source]

Stop the service.

Return type:None
coroutine transition_with(self, flag: str, fut: Awaitable, *args, **kwargs) → Any[source]
Return type:Any
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_for_stopped(self, *coros, timeout: Union[datetime.timedelta, float, str] = None) → bool[source]
Return type:bool
coroutine wait_many(self, coros: Iterable[Union[typing.Generator[typing.Any, NoneType, typing.Any], typing.Awaitable, asyncio.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
service_reset() → None[source]
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
started

Was the service started? :rtype: bool

crashed
should_stop

Should the service stop ASAP? :rtype: bool

state

Current service state - as a human readable string. :rtype: str

label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

beacon

Beacon used to track services in a dependency graph. :rtype: NodeT[~_T]

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

mode.Service.

wait_for_shutdown = False
restart_count = 0
supervisor = None
add_dependency(service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]
Return type:ServiceT[]
add_context(context: ContextManager) → Any[source]
Return type:Any
service_reset() → None[source]
Return type:None
set_shutdown() → None[source]
Return type:None
started
crashed
should_stop
state
label
shortlabel
coroutine add_async_context(self, context: AsyncContextManager) → Any[source]
Return type:Any
coroutine add_runtime_dependency(self, service: mode.types.services.ServiceT) → mode.types.services.ServiceT[source]
Return type:ServiceT[]
beacon
coroutine crash(self, reason: BaseException) → None[source]
Return type:None
coroutine maybe_start(self) → None[source]
Return type:None
coroutine restart(self) → None[source]
Return type:None
coroutine start(self) → None[source]
Return type:None
coroutine stop(self) → None[source]
Return type:None
coroutine wait_until_stopped(self) → None[source]
Return type:None
faust.exceptions

Faust exceptions.

exception faust.exceptions.FaustError[source]

Base-class for all Faust exceptions.

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.KeyDecodeError[source]

Error while decoding/deserializing message key.

exception faust.exceptions.ValueDecodeError[source]

Error while decoding/deserializing message value.

faust.app
class faust.app.App(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = 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.

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).

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)
Parameters:
  • silent (bool) – If true then import errors will be ignored.
  • force (bool) – Force reading configuration immediately. By default the configuration will be read only when required.
Return type:

None

finalize() → None[source]
Return type:None
worker_init() → None[source]
Return type:None
discover(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[str] = ['test_.*', '.*__main__.*']) → None[source]
Return type:None
main() → None[source]

Execute the faust umbrella command using this app.

Return type:None
topic(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, 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().

Return type:TopicT[]
channel(*, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, maxsize: int = 1, 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).

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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
task(fun: Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.Awaitable]]) → Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.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')
Return type:Union[Callable[[AppT[]], Awaitable[+T_co]], Callable[[], Awaitable[+T_co]]]
timer(interval: Union[datetime.timedelta, float, str], on_leader: bool = False) → 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:Callable
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):
    ...
Return type:Type[ServiceT[]]
is_leader() → bool[source]
Return type:bool
stream(channel: Union[typing.AsyncIterable, typing.Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]

Create new stream from channel/topic/iterable/async iterable.

Parameters:
Return type:

StreamT[+T_co]

Returns:

to iterate over events in the stream.

Return type:

faust.Stream

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[]
page(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>) → Callable[Union[typing.Type[faust.types.web.View], typing.Callable[[faust.types.web.View, faust.types.web.Request], typing.Awaitable[faust.types.web.Response]]], Type[faust.web.views.Site]][source]
Return type:Callable[[Union[Type[View], Callable[[View, Request], Awaitable[Response]]]], Type[Site]]
table_route(table: faust.types.tables.CollectionT, shard_param: str) → Callable[Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]], Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]]][source]
Return type:Callable[[Callable[[View, Request], Awaitable[Response]]], Callable[[View, Request], Awaitable[Response]]]
command(*options, base: Union[typing.Type[faust.app.base.AppCommand], NoneType] = None, **kwargs) → Callable[Callable, Type[faust.app.base.AppCommand]][source]
Return type:Callable[[Callable], Type[AppCommand]]
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:ThrowableQueue
Worker(**kwargs) → faust.app.base.WorkerT[source]
Return type:WorkerT
on_webserver_init(web: faust.web.base.Web) → None[source]
Return type:None
conf
producer[source]
consumer
transport

Message transport. :rtype: TransportT

tables[source]

Map of available tables, and the table manager service.

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:bool
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
maybe_start_producer[source]

Ensure producer is started.

coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine send(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send event to channel/topic.

Parameters:
Return type:

Awaitable[RecordMetadata]

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
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.

monitor

Monitor keeps stats about what’s going on inside the worker. :rtype: Monitor[]

flow_control[source]

Internal flow control.

This object controls flow into stream queues, and can also clear all buffers.

http_client

HTTP Client Session. :rtype: HttpClientT

assignor[source]

Partition Assignor.

Responsible for partition assignment.

router[source]

Find the node partitioned data belongs to.

The router helps us route web requests to the wanted Faust node. If a topic is sharded by account_id, the router can send us to the Faust worker responsible for any account. Used by the @app.table_route decorator.

serializers[source]
label
shortlabel
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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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 new put call will block until a message is removed from the channel.
  • loop (Optional[AbstractEventLoop]) – The asyncio event loop to use.
coroutine deliver(self, message: faust.types.tuples.Message) → None[source]
Return type:None
queue
clone(*, is_iterator: bool = None, **kwargs) → 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]
get_topic_name() → str[source]
Return type:str
as_future_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → faust.types.tuples.FutureMessage[source]
Return type:FutureMessage[]
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
empty() → bool[source]
Return type:bool
coroutine declare(self) → None[source]
Return type:None
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:Any
maybe_declare[source]
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
on_stop_iteration() → 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:Awaitable[RecordMetadata]
coroutine put(self, value: Any) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send message to channel.

Return type:Awaitable[RecordMetadata]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
subscriber_count
label
faust.events
class faust.events.App[source]
class faust.events.Event(app: faust.types.app.AppT, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], 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

  • Iteratiing 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, typing.Any, NoneType] = <object object>, value: Union[bytes, faust.types.core.ModelT, typing.Any] = <object object>, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Forward original message (will not be reserialized).

Return type:Awaitable[RecordMetadata]
coroutine send(self, channel: Union[str, faust.types.channels.ChannelT], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = <object object>, value: Union[bytes, faust.types.core.ModelT, typing.Any] = <object object>, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send object to channel.

Return type:Awaitable[RecordMetadata]
ack() → bool[source]
Return type:bool
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.

coroutine process(self, event: faust.types.events.EventT) → Union[faust.types.events.EventT, NoneType][source]
Return type:Optional[EventT[]]
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.

class faust.joins.InnerJoin(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]

Inner-join strategy.

class faust.joins.OuterJoin(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]

Outer-join strategy.

faust.streams

Streams.

faust.streams.current_event() → Union[faust.types.events.EventT, NoneType][source]

Return the event currently being processed, or None.

Return type:Optional[EventT[]]
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, loop: asyncio.events.AbstractEventLoop = None) → None[source]

A stream: async iterator processing events in channels/topics.

logger = <Logger faust.streams (WARNING)>
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 be c.

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]
get_root_stream() → faust.types.streams.StreamT[source]
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
info() → Mapping[str, Any][source]

Return stream settings as a dictionary.

Return type:Mapping[str, Any]
clone(**kwargs) → faust.types.streams.StreamT[source]

Create a clone of this stream.

Notes

If the cloned stream is supposed to “supercede” this stream, like in group_by/through/etc., you should use _chain() instead so stream._next = cloned_stream is set and get_active_stream() returns the cloned stream.

Return type:StreamT[+T_co]
noack() → faust.types.streams.StreamT[source]
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:AsyncIterable[EventT[]]
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, typing.Callable[[~T], typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType]]], *, 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.
  • 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[]

combine(*nodes, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
contribute_to_stream(active: faust.types.streams.StreamT) → None[source]
Return type:None
join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
left_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
inner_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
outer_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
coroutine on_merge(self, value: T = None) → Union[~T, NoneType][source]
Return type:Optional[~T]
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.

Parameters:event (EventT[]) – Event to ack.
Return type:bool
items() → AsyncIterator[Tuple[Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], 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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
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
coroutine stop(self) → None[source]

Stop the service.

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]]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel[source]
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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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]

Define new topic description.

Parameters:
  • app (AppT[]) – App instance used to create this topic description.
  • topics (Optional[Sequence[str]]) – List of topic names.
  • 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, and autoCreateTopics 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 a faust.Model type, str, bytes, or None for “autodetect”
  • value_type (Union[Type[ModelT], Type[bytes], Type[str], None]) – How to deserialize values for messages in this topic. Can be a faust.Model type, str, bytes, or None for “autodetect”
  • active_partitions (Optional[Set[TP]]) – Set of faust.types.tuples.TP that this topic should be restricted to.
Raises:

TypeError – if both topics and pattern is provided.

pattern
derive(**kwargs) → faust.types.channels.ChannelT[source]

Create new Topic derived from this topic.

Configuration will be copied from this topic, but any parameter overriden 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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[]
get_topic_name() → str[source]
Return type:str
coroutine declare(self) → None[source]
Return type:None
coroutine decode(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.events.EventT[source]
Return type:EventT[]
maybe_declare[source]
coroutine publish_message(self, fut: faust.types.tuples.FutureMessage, wait: bool = True) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine put(self, event: faust.types.events.EventT) → None[source]
Return type:None
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
on_stop_iteration() → None[source]
Return type:None
partitions
faust.windows

Window Types.

class faust.windows.Window(*args, **kwargs)[source]
class faust.windows.HoppingWindow(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → None[source]

Hopping window type.

Fixed-size, overlapping windows.

ranges(timestamp: float) → List[faust.types.windows.WindowRange][source]
Return type:List[WindowRange]
stale(timestamp: float, latest_timestamp: float) → bool[source]
Return type:bool
current(timestamp: float) → faust.types.windows.WindowRange[source]
Return type:WindowRange
delta(timestamp: float, d: Union[datetime.timedelta, float, str]) → faust.types.windows.WindowRange[source]
Return type:WindowRange
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.

class faust.windows.SlidingWindow(before: Union[datetime.timedelta, float, str], after: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str]) → None[source]

Sliding window type.

Fixed-size, overlapping windows that work on differences between record timestamps

ranges(timestamp: float) → List[faust.types.windows.WindowRange][source]

Return list of windows from timestamp.

Notes

SELECT * FROM s1, s2
WHERE
    s1.key = s2.key
AND
s1.ts - before <= s2.ts AND s2.ts <= s1.ts + after
Return type:List[WindowRange]
stale(timestamp: float, latest_timestamp: float) → bool[source]
Return type:bool
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, typing.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, Website: Union[typing.Type[faust.web.site.Website], str] = 'faust.web.site:Website', web_port: int = None, web_bind: str = None, web_host: str = None, console_port: int = 50101, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]

Worker.

Usage:

You can start a worker using:

  1. the faust worker program.

  2. instantiating Worker programmatically and calling execute_from_commandline():

    >>> worker = Worker(app)
    >>> worker.execute_from_commandline()
    
  3. 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.
  • Website (Union[Type[Website[]]]) – Class used to serve the Faust web site (defaults to faust.web.site.Website).
  • web_port (int) – Port for web site to bind to (defaults to 6066).
  • web_bind (str) – Host to bind web site to (defaults to “0.0.0.0”).
  • web_host (str) – Canonical host name used for this server. (defaults to the current host name).
  • 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.

Website = None

Class that starts our web server and serves the Faust website.

web_port = None

Port to run the embedded web server on (defaults to 6066).

web_bind = None

Host to bind web server port to (defaults to ‘0.0.0.0’).

spinner = None

Class that displays a terminal progress spinner (see progress).

on_init_dependencies() → Iterable[mode.types.services.ServiceT][source]

Callback to be used to add service dependencies.

Return type:Iterable[ServiceT[]]
change_workdir(path: pathlib.Path) → None[source]
Return type:None
autodiscover() → None[source]
Return type:None
coroutine on_execute(self) → None[source]
Return type:None
coroutine on_first_start(self) → None[source]

Called only the first time the service is started.

Return type:None
coroutine on_startup_finished(self) → None[source]
Return type:None
on_setup_root_logger(logger: logging.Logger, level: int) → None[source]
Return type:None
website[source]

App

faust.app.base

Faust Application.

An app is an instance of the Faust library. Everything starts here.

class faust.app.base.App(id: str, *, monitor: faust.sensors.monitor.Monitor = None, config_source: Any = None, loop: asyncio.events.AbstractEventLoop = 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.

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).

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)
Parameters:
  • silent (bool) – If true then import errors will be ignored.
  • force (bool) – Force reading configuration immediately. By default the configuration will be read only when required.
Return type:

None

finalize() → None[source]
Return type:None
worker_init() → None[source]
Return type:None
discover(*extra_modules, categories: Iterable[str] = ['faust.agent', 'faust.command', 'faust.page', 'faust.service', 'faust.task'], ignore: Iterable[str] = ['test_.*', '.*__main__.*']) → None[source]
Return type:None
main() → None[source]

Execute the faust umbrella command using this app.

Return type:None
topic(*topics, pattern: Union[str, Pattern[~AnyStr]] = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, 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().

Return type:TopicT[]
channel(*, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, maxsize: int = 1, 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).

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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.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
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
task(fun: Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.Awaitable]]) → Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.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')
Return type:Union[Callable[[AppT[]], Awaitable[+T_co]], Callable[[], Awaitable[+T_co]]]
timer(interval: Union[datetime.timedelta, float, str], on_leader: bool = False) → 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:Callable
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):
    ...
Return type:Type[ServiceT[]]
is_leader() → bool[source]
Return type:bool
stream(channel: Union[typing.AsyncIterable, typing.Iterable], beacon: mode.utils.types.trees.NodeT = None, **kwargs) → faust.types.streams.StreamT[source]

Create new stream from channel/topic/iterable/async iterable.

Parameters:
Return type:

StreamT[+T_co]

Returns:

to iterate over events in the stream.

Return type:

faust.Stream

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[]
page(path: str, *, base: Type[faust.web.views.View] = <class 'faust.web.views.View'>) → Callable[Union[typing.Type[faust.types.web.View], typing.Callable[[faust.types.web.View, faust.types.web.Request], typing.Awaitable[faust.types.web.Response]]], Type[faust.web.views.Site]][source]
Return type:Callable[[Union[Type[View], Callable[[View, Request], Awaitable[Response]]]], Type[Site]]
table_route(table: faust.types.tables.CollectionT, shard_param: str) → Callable[Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]], Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]]][source]
Return type:Callable[[Callable[[View, Request], Awaitable[Response]]], Callable[[View, Request], Awaitable[Response]]]
command(*options, base: Union[typing.Type[faust.app.base.AppCommand], NoneType] = None, **kwargs) → Callable[Callable, Type[faust.app.base.AppCommand]][source]
Return type:Callable[[Callable], Type[AppCommand]]
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:ThrowableQueue
Worker(**kwargs) → faust.app.base.WorkerT[source]
Return type:WorkerT
on_webserver_init(web: faust.web.base.Web) → None[source]
Return type:None
conf
producer[source]
consumer
transport

Message transport. :rtype: TransportT

tables[source]

Map of available tables, and the table manager service.

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:bool
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
maybe_start_producer[source]

Ensure producer is started.

coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine send(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]

Send event to channel/topic.

Parameters:
Return type:

Awaitable[RecordMetadata]

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
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.

monitor

Monitor keeps stats about what’s going on inside the worker. :rtype: Monitor[]

flow_control[source]

Internal flow control.

This object controls flow into stream queues, and can also clear all buffers.

http_client

HTTP Client Session. :rtype: HttpClientT

assignor[source]

Partition Assignor.

Responsible for partition assignment.

router[source]

Find the node partitioned data belongs to.

The router helps us route web requests to the wanted Faust node. If a topic is sharded by account_id, the router can send us to the Faust worker responsible for any account. Used by the @app.table_route decorator.

serializers[source]
label
shortlabel
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, typing.Any, NoneType]) → yarl.URL[source]
Return type:URL
table_metadata(table_name: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
tables_metadata() → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
coroutine route_req(self, table_name: str, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], web: faust.types.web.Web, request: faust.types.web.Request) → faust.types.web.Response[source]
Return type:Response
faust.app.service
class faust.app.service.App[source]
class faust.app.service.AppService(app: faust.app.service.App, **kwargs) → None[source]

Service responsible for starting/stopping an application.

on_init_dependencies() → Iterable[mode.types.services.ServiceT][source]

Callback to be used to add service dependencies.

Return type:Iterable[ServiceT[]]
logger = <Logger faust.app.service (WARNING)>
coroutine on_first_start(self) → None[source]

Called only the first time the service is started.

Return type:None
coroutine on_init_extra_service(self, service: Union[mode.types.services.ServiceT, typing.Type[mode.types.services.ServiceT]]) → mode.types.services.ServiceT[source]
Return type:ServiceT[]
coroutine on_restart(self) → None[source]

Called every time when the service is restarted.

Return type:None
coroutine on_shutdown(self) → None[source]

Called every time after the service is stopped/restarted

Return type:None
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_started(self) → None[source]

Called every time after the service is started/restarted.

Return type:None
coroutine on_started_init_extra_services(self) → None[source]
Return type:None
coroutine on_started_init_extra_tasks(self) → None[source]
Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine wait_for_table_recovery_completed(self) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

Agents

faust.agents
class faust.agents.Agent(fun: Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, on_error: Callable[[_ForwardRef('AgentT'), BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, isolated_partitions: bool = False, **kwargs) → None[source]

Agent.

This is the type of object returned by the @app.agent decorator.

cancel() → None[source]
Return type:None
info() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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, **kwargs) → faust.types.agents.AgentTestWrapperT[source]
Return type:AgentTestWrapperT[]
actor_from_stream(stream: faust.types.streams.StreamT) → faust.types.agents.ActorT[Union[typing.AsyncIterable, typing.Awaitable]][source]
Return type:ActorT[]
add_sink(sink: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]) → None[source]
Return type:None
stream(active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
get_topic_names() → Iterable[str][source]
Return type:Iterable[str]
coroutine ask(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]
Return type:Any
coroutine ask_nowait(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]
Return type:ReplyPromise
coroutine cast(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None) → None[source]
Return type:None
channel
coroutine join(self, values: Union[typing.AsyncIterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]], typing.Iterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvjoin(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
kvmap(items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]
Return type:AsyncIterator[str]
logger = <Logger faust.agents.agent (WARNING)>
map(values: Union[typing.AsyncIterable, typing.Iterable], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('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
coroutine on_shared_partitions_assigned(self, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_shared_partitions_revoked(self, revoked: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, *, reply_to: Union[_ForwardRef('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:Awaitable[RecordMetadata]
channel_iterator
label
faust.agents.AgentFun

alias of typing.Callable

class faust.agents.AgentT(fun: Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], *, name: str = None, app: faust.types.agents.AppT = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, on_error: Callable[[_ForwardRef('AgentT'), BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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:AgentTestWrapperT[]
add_sink(sink: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]) → None[source]
Return type:None
stream(**kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
info() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
clone(*, cls: Type[_ForwardRef('AgentT')] = None, **kwargs) → faust.types.agents.AgentT[source]
Return type:AgentT[]
get_topic_names() → Iterable[str][source]
Return type:Iterable[str]
channel
channel_iterator
coroutine ask(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]
Return type:Any
coroutine cast(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None) → None[source]
Return type:None
coroutine join(self, values: Union[typing.AsyncIterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]], typing.Iterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvjoin(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvmap(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]
coroutine map(self, values: Union[typing.AsyncIterable, typing.Iterable], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('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, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, *, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
class faust.agents.AgentManager(app: faust.types.app.AppT, **kwargs) → None[source]

Agent manager.

update_topic_index() → None[source]
Return type:None
logger = <Logger faust.agents.manager (WARNING)>
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
class faust.agents.AgentManagerT(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[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
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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
faust.agents.actor
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.

cancel() → None[source]
Return type:None
label

Label used for graphs. :rtype: str

logger = <Logger faust.agents.actor (WARNING)>
coroutine on_isolated_partition_assigned(self, tp: faust.types.tuples.TP) → None[source]
Return type:None
coroutine on_isolated_partition_revoked(self, tp: faust.types.tuples.TP) → None[source]
Return type:None
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

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[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], *, app: faust.types.app.AppT, name: str = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, on_error: Callable[[_ForwardRef('AgentT'), BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, isolated_partitions: bool = False, **kwargs) → None[source]

Agent.

This is the type of object returned by the @app.agent decorator.

cancel() → None[source]
Return type:None
info() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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, **kwargs) → faust.types.agents.AgentTestWrapperT[source]
Return type:AgentTestWrapperT[]
actor_from_stream(stream: faust.types.streams.StreamT) → faust.types.agents.ActorT[Union[typing.AsyncIterable, typing.Awaitable]][source]
Return type:ActorT[]
add_sink(sink: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]) → None[source]
Return type:None
stream(active_partitions: Set[faust.types.tuples.TP] = None, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
get_topic_names() → Iterable[str][source]
Return type:Iterable[str]
coroutine ask(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]
Return type:Any
coroutine ask_nowait(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None, force: bool = False) → faust.agents.replies.ReplyPromise[source]
Return type:ReplyPromise
coroutine cast(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None) → None[source]
Return type:None
channel
coroutine join(self, values: Union[typing.AsyncIterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]], typing.Iterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvjoin(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
kvmap(items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]
Return type:AsyncIterator[str]
logger = <Logger faust.agents.agent (WARNING)>
map(values: Union[typing.AsyncIterable, typing.Iterable], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('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
coroutine on_shared_partitions_assigned(self, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_shared_partitions_revoked(self, revoked: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, *, reply_to: Union[_ForwardRef('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:Awaitable[RecordMetadata]
channel_iterator
label
faust.agents.manager

Agent manager.

class faust.agents.manager.AgentManager(app: faust.types.app.AppT, **kwargs) → None[source]

Agent manager.

update_topic_index() → None[source]
Return type:None
logger = <Logger faust.agents.manager (WARNING)>
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
class faust.agents.manager.AgentManagerService(agents: faust.agents.manager.AgentManager, **kwargs) → None[source]
service_reset() → None[source]
Return type:None
cancel() → None[source]
Return type:None
logger = <Logger faust.agents.manager (WARNING)>
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine stop(self) → None[source]

Stop the service.

Return type:None
faust.agents.models

Models used by agents internally.

class faust.agents.models.ReqRepRequest(value, reply_to, correlation_id, *, __strict__=True, __faust=None, **kwargs) → NoneType[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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
asdict()
class faust.agents.models.ReqRepResponse(key, value, correlation_id, *, __strict__=True, __faust=None, **kwargs) → NoneType[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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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.

fulfill(correlation_id: str, value: Any) → None[source]
Return type:None
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.

add(p: faust.agents.replies.ReplyPromise) → None[source]
Return type:None
finalize() → None[source]
Return type:None
fulfill(correlation_id: str, value: Any) → None[source]
Return type:None
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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None

Fixups

faust.fixups

Transport registry.

faust.fixups.fixups(app: faust.types.fixups.AppT) → Iterator[faust.types.fixups.FixupT][source]
Return type:Iterator[FixupT]
faust.fixups.base
class faust.fixups.base.Fixup(app: faust.types.fixups.AppT) → None[source]
enabled() → bool[source]
Return type:bool
autodiscover_modules() → Iterable[str][source]
Return type:Iterable[str]
on_worker_init() → None[source]
Return type:None
faust.fixups.django
class faust.fixups.django.Fixup(app: faust.types.fixups.AppT) → None[source]
enabled() → bool[source]
Return type:bool
autodiscover_modules() → Iterable[str][source]
Return type:Iterable[str]
on_worker_init() → None[source]
Return type:None
apps[source]
settings[source]

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'>, '@TableInfo': <class 'faust.web.apps.tables.TableInfo'>}

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, NoneType] = None) → faust.types.models.ModelT[source]

Deserialize model object from bytes.

Parameters:
  • default_serializer (CodecArg) – Default serializer to use if no custom serializer was set for this model subclass.
  • **kwargs – Additional attributes to set on the model object. Note, these are regarded as defaults, and any fields also present in the message takes precedence.
Return type:

ModelT

to_representation() → Any[source]

Convert object to JSON serializable object.

Return type:Any
derive(*objects, **fields) → faust.types.models.ModelT[source]
Return type:ModelT
dumps(*, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → bytes[source]

Serialize object to the target serialization format.

Return type:bytes
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 (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
field = None

Name of attribute on Model.

type = None

Type of value (e.g. int, or Optional[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.

getattr(obj: faust.types.models.ModelT) → Any[source]
Return type:Any
ident
faust.models.record

Record - Dictionary Model.

class faust.models.record.Record(*args, __strict__: bool = True, _Record__faust: Any = None, **kwargs) → 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:Record
to_representation() → Mapping[str, Any][source]

Convert object to JSON serializable object.

Return type:Mapping[str, Any]
asdict() → Dict[str, Any][source]
Return type:Dict[str, Any]

Sensors

faust.sensors
class faust.sensors.Monitor(*, max_avg_history: int = 100, max_commit_latency_history: int = 30, max_send_latency_history: int = 30, 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: List[float] = None, commit_latency: List[float] = None, send_latency: List[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, **kwargs) → None[source]

Default Faust Sensor.

This is the default sensor, recording statistics about events, etc.

max_avg_history = 0

Max number of total run time values to keep to build average.

max_commit_latency_history = 0

Max number of commit latency numbers to keep.

max_send_latency_history = 0

Max number of send latency numbers to keep.

tables = None

Mapping of tables

commit_latency = None

List of commit latency values

send_latency = None

List of send latency values

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

List 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

asdict() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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_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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

Return type:None
count(metric_name: str, count: int = 1) → None[source]
Return type:None
on_tp_commit(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]
Return type:None
track_tp_end_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

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.

add(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
remove(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
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) → None[source]
Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → None[source]
Return type:None
on_topic_buffer_full(topic: faust.types.topics.TopicT) → 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_get(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_table_set(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]
Return type:None
on_table_del(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_commit_initiated(consumer: faust.types.transports.ConsumerT) → Any[source]
Return type:Any
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, keysize: int, valsize: int) → Any[source]
Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]
Return type:None
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.

asdict() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

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.

add(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
remove(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
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) → None[source]
Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → None[source]
Return type:None
on_topic_buffer_full(topic: faust.types.topics.TopicT) → 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_get(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_table_set(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]
Return type:None
on_table_del(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_commit_initiated(consumer: faust.types.transports.ConsumerT) → Any[source]
Return type:Any
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, keysize: int, valsize: int) → Any[source]
Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]
Return type:None
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.

asdict() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
class faust.sensors.monitor.Monitor(*, max_avg_history: int = 100, max_commit_latency_history: int = 30, max_send_latency_history: int = 30, 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: List[float] = None, commit_latency: List[float] = None, send_latency: List[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, **kwargs) → None[source]

Default Faust Sensor.

This is the default sensor, recording statistics about events, etc.

max_avg_history = 0

Max number of total run time values to keep to build average.

max_commit_latency_history = 0

Max number of commit latency numbers to keep.

max_send_latency_history = 0

Max number of send latency numbers to keep.

tables = None

Mapping of tables

commit_latency = None

List of commit latency values

send_latency = None

List of send latency values

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

List 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

asdict() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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_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:Any
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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

Return type:None
count(metric_name: str, count: int = 1) → None[source]
Return type:None
on_tp_commit(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]
Return type:None
track_tp_end_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
class faust.sensors.monitor.MonitorService(monitor: faust.sensors.monitor.Monitor, **kwargs) → None[source]

Service responsible for starting/stopping a sensor.

logger = <Logger faust.sensors.monitor (WARNING)>
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) → None[source]

Message sent to a stream as an event.

Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → 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_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, keysize: int, valsize: int) → Any[source]

About to send a message.

Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]

Message successfully sent.

Return type:None
count(metric_name: str, count: int = 1) → None[source]
Return type:None
on_tp_commit(tp_offsets: MutableMapping[faust.types.tuples.TP, int]) → None[source]
Return type:None
track_tp_end_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
logger = <Logger faust.sensors.statsd (WARNING)>
client[source]

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 func:dumps function takes a codec name and the object to encode, then returns bytes:

>>> s = dumps('json', obj)

For the reverse direction, the func: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 ur 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 entrypoints, 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. 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.

kwargs = None

subclasses can support keyword arguments, the base implementation of clone() uses this to preserve keyword arguments in copies.

dumps(obj: Any) → bytes[source]

Encode object obj.

Return type:bytes
loads(s: bytes) → Any[source]

Decode object from string.

Return type:Any
clone(*children) → faust.types.codecs.CodecT[source]

Create a clone of this codec, with optional children added.

Return type:CodecT
faust.serializers.codecs.register(name: str, codec: faust.types.codecs.CodecT) → None[source]

Register new codec in the codec registy.

Return type:None
faust.serializers.codecs.get_codec(name_or_codec: Union[faust.types.codecs.CodecT, str, NoneType]) → faust.types.codecs.CodecT[source]

Get codec by name.

Return type:CodecT
faust.serializers.codecs.dumps(codec: Union[faust.types.codecs.CodecT, str, NoneType], obj: Any) → bytes[source]

Encode object into bytes.

Return type:bytes
faust.serializers.codecs.loads(codec: Union[faust.types.codecs.CodecT, str, NoneType], s: bytes) → Any[source]

Decode object from bytes.

Return type:Any
faust.serializers.registry

Registry of supported codecs (serializers, compressors, etc.).

class faust.serializers.registry.Registry(key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json') → None[source]

Serializing message keys/values.

Parameters:
  • key_serializer (Union[CodecT, str, None]) – Default key serializer to use when none provided.
  • value_serializer (Union[CodecT, str, None]) – Default value serializer to use when none provided.
loads_key(typ: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str], NoneType], key: Union[bytes, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Union[bytes, faust.types.core.ModelT, typing.Any, NoneType][source]

Deserialize message key.

Parameters:
Return type:

Union[bytes, ModelT, Any, None]

loads_value(typ: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str], NoneType], value: Union[bytes, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Any[source]

Deserialize value.

Parameters:
Return type:

Any

dumps_key(typ: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str], NoneType], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, skip: Tuple[Type, ...] = (<class 'bytes'>,)) → Union[bytes, NoneType][source]

Serialize key.

Parameters:
  • typ (Union[Type[ModelT], Type[bytes], Type[str], None]) – Model hint (can also be str/bytes). When typ=str or bytes, 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:

Optional[bytes]

dumps_value(typ: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str], NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, skip: Tuple[Type, ...] = (<class 'bytes'>,)) → Union[bytes, NoneType][source]

Serialize value.

Parameters:
  • typ (Union[Type[ModelT], Type[bytes], Type[str], None]) – Model hint (can also be str/bytes). When typ=str or bytes, 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:

Optional[bytes]

Model[source]

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_name: str = '', key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', **kwargs) → None[source]

Base class for table storage drivers.

persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
set_persisted_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

logger = <Logger faust.stores.base (WARNING)>
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_partitions_assigned(self, table: faust.types.tables.CollectionT, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_partitions_revoked(self, table: faust.types.tables.CollectionT, revoked: Set[faust.types.tuples.TP]) → None[source]
Return type:None
class faust.stores.base.SerializedStore(url: Union[str, yarl.URL], app: faust.types.app.AppT, *, table_name: str = '', key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', **kwargs) → None[source]

Base class for table storage drivers requiring serialization.

apply_changelog_batch(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, Any], to_value: Callable[Any, Any]) → None[source]
Return type:None
keys() → collections.abc.KeysView[source]
Return type:KeysView
values() → collections.abc.ValuesView[source]
Return type:ValuesView
items() → collections.abc.ItemsView[source]
Return type:ItemsView
clear() → 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_name: str = '', key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json', **kwargs) → None[source]

Table storage using an in-memory dictionary.

on_init() → None[source]
Return type:None
apply_changelog_batch(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, Any], to_value: Callable[Any, Any]) → None[source]
Return type:None
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
reset_state() → None[source]
Return type:None
logger = <Logger faust.stores.memory (WARNING)>
faust.stores.rocksdb

RocksDB storage.

class faust.stores.rocksdb.DB[source]

Dummy DB.

class faust.stores.rocksdb.Options[source]

Dummy Options.

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.

bloom_filter_size = 3
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
open(path: pathlib.Path, *, read_only: bool = False) → faust.stores.rocksdb.DB[source]
Return type:DB
as_options() → faust.stores.rocksdb.Options[source]
Return type:Options
class faust.stores.rocksdb.Store(url: Union[str, yarl.URL], app: faust.types.app.AppT, *, 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).

persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
set_persisted_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
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.rocksdb (WARNING)>
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_partitions_assigned(self, table: faust.types.tables.CollectionT, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_partitions_revoked(self, table: faust.types.tables.CollectionT, revoked: Set[faust.types.tuples.TP]) → None[source]
Return type:None
reset_state() → None[source]
Return type:None
partition_path(partition: int) → pathlib.Path[source]
Return type:Path
with_suffix(path: pathlib.Path, *, suffix: str = '.db') → pathlib.Path[source]
Return type:Path
path
basename

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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = 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[NoneType]]) → Callable[Awaitable[NoneType]][source]

Add function as callback to be called on table recovery.

Return type:Callable[[], Awaitable[None]]
info() → Mapping[str, Any][source]
Return type:Mapping[str, Any]
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
reset_state() → None[source]
Return type:None
join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
left_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
inner_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
outer_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
clone(**kwargs) → Any[source]
Return type:Any
combine(*nodes, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
contribute_to_stream(active: faust.types.streams.StreamT) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

coroutine call_recover_callbacks(self) → None[source]
Return type:None
logger = <Logger faust.tables.base (WARNING)>
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_changelog_event(self, event: faust.types.events.EventT) → 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
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine remove_from_stream(self, stream: faust.types.streams.StreamT) → None[source]
Return type:None
changelog_topic
apply_changelog_batch(batch: Iterable[faust.types.events.EventT]) → 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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]
StateStore = None
changelog_topic
apply_changelog_batch(batch: Iterable[faust.types.events.EventT]) → None[source]
Return type:None
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
reset_state() → None[source]
Return type:None
on_recover(fun: Callable[Awaitable[NoneType]]) → Callable[Awaitable[NoneType]][source]
Return type:Callable[[], Awaitable[None]]
coroutine call_recover_callbacks(self) → None[source]
Return type:None
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_changelog_event(self, event: faust.types.events.EventT) → 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
class faust.tables.TableManager(app: faust.types.app.AppT, **kwargs) → None[source]

Manage tables used by Faust worker.

changelog_topics
add(table: faust.types.tables.CollectionT) → faust.types.tables.CollectionT[source]
Return type:CollectionT[]
logger = <Logger faust.tables.manager (WARNING)>
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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
class faust.tables.TableManagerT(app: faust.types.tables.AppT, **kwargs) → None[source]
add(table: faust.types.tables.CollectionT) → faust.types.tables.CollectionT[source]
Return type:CollectionT[]
changelog_topics
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
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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]

Table (non-windowed).

using_window(window: faust.types.windows.WindowT) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
hopping(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
tumbling(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
on_key_get(key: Any) → None[source]

Handle that key is being retrieved.

Return type:None
on_key_set(key: Any, value: Any) → None[source]

Handle that value for a key is being set.

Return type:None
on_key_del(key: Any) → None[source]

Handle that a key is deleted.

Return type:None
as_ansitable(*, key: str = 'Key', value: str = 'Value', sort: bool = False, sortkey: Callable[Any, Any] = operator.itemgetter(0), target: IO = <_io.TextIOWrapper name='<stdout>' mode='w' encoding='UTF-8'>, title: str = '{table.name}') → str[source]
Return type:str
logger = <Logger faust.tables.table (WARNING)>
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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = 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) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
hopping(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
tumbling(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
as_ansitable(*, key: str = 'Key', value: str = 'Value', sort: bool = False, sortkey: Callable[Any, Any] = <function TableT.<lambda>>, title: str = 'Title') → str[source]
Return type:str
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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = 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[NoneType]]) → Callable[Awaitable[NoneType]][source]

Add function as callback to be called on table recovery.

Return type:Callable[[], Awaitable[None]]
info() → Mapping[str, Any][source]
Return type:Mapping[str, Any]
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
reset_state() → None[source]
Return type:None
join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
left_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
inner_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
outer_join(*fields) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
clone(**kwargs) → Any[source]
Return type:Any
combine(*nodes, **kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
contribute_to_stream(active: faust.types.streams.StreamT) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

coroutine call_recover_callbacks(self) → None[source]
Return type:None
logger = <Logger faust.tables.base (WARNING)>
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_changelog_event(self, event: faust.types.events.EventT) → 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
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine remove_from_stream(self, stream: faust.types.streams.StreamT) → None[source]
Return type:None
changelog_topic
apply_changelog_batch(batch: Iterable[faust.types.events.EventT]) → None[source]
Return type:None
faust.tables.changelogs
class faust.tables.changelogs.ChangelogReader(table: faust.types.tables.CollectionT, channel: faust.types.channels.ChannelT, app: faust.types.app.AppT, tps: Set[faust.types.tuples.TP], offsets: Counter[faust.types.tuples.TP] = None, stats_interval: Union[datetime.timedelta, float, str] = 5.0, **kwargs) → None[source]

Service synchronizing table state from changelog topic.

wait_for_shutdown = True
shutdown_timeout = None
recovered() → bool[source]
Return type:bool
logger = <Logger faust.tables.changelogs (WARNING)>
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine wait_done_reading(self) → None[source]
Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

coroutine faust.tables.changelogs.local_tps(table: faust.types.tables.CollectionT, tps: Iterable[faust.types.tuples.TP]) → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
class faust.tables.changelogs.StandbyReader(table: faust.types.tables.CollectionT, channel: faust.types.channels.ChannelT, app: faust.types.app.AppT, tps: Set[faust.types.tuples.TP], offsets: Counter[faust.types.tuples.TP] = None, stats_interval: Union[datetime.timedelta, float, str] = 5.0, **kwargs) → None[source]

Service reading table changelogs to keep an up-to-date backup.

logger = <Logger faust.tables.changelogs (WARNING)>
recovered() → bool[source]
Return type:bool
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.

changelog_topics
add(table: faust.types.tables.CollectionT) → faust.types.tables.CollectionT[source]
Return type:CollectionT[]
logger = <Logger faust.tables.manager (WARNING)>
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 on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]

Table (non-windowed).

using_window(window: faust.types.windows.WindowT) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
hopping(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
tumbling(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
on_key_get(key: Any) → None[source]

Handle that key is being retrieved.

Return type:None
on_key_set(key: Any, value: Any) → None[source]

Handle that value for a key is being set.

Return type:None
on_key_del(key: Any) → None[source]

Handle that a key is deleted.

Return type:None
as_ansitable(*, key: str = 'Key', value: str = 'Value', sort: bool = False, sortkey: Callable[Any, Any] = operator.itemgetter(0), target: IO = <_io.TextIOWrapper name='<stdout>' mode='w' encoding='UTF-8'>, title: str = '{table.name}') → str[source]
Return type:str
logger = <Logger faust.tables.table (WARNING)>
faust.tables.wrappers

Wrappers for windowed tables.

class faust.tables.wrappers.WindowSet(key: Any, 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 since k 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[[Any, Any], Any], value: Any, event: faust.types.events.EventT = None) → faust.types.tables.WindowSetT[source]
Return type:WindowSetT[]
value(event: faust.types.events.EventT = None) → Any[source]
Return type:Any
now() → Any[source]
Return type:Any
current(event: faust.types.events.EventT = None) → Any[source]
Return type:Any
delta(d: Union[datetime.timedelta, float, str], event: faust.types.events.EventT = None) → Any[source]
Return type:Any
class faust.tables.wrappers.WindowWrapper(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables.FieldDescriptorT, typing.Callable[[typing.Union[faust.types.events.EventT, NoneType]], typing.Union[float, datetime.datetime]], datetime.datetime, float, NoneType] = 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.

clone(relative_to: Union[faust.types.tables.FieldDescriptorT, typing.Callable[[typing.Union[faust.types.events.EventT, NoneType]], typing.Union[float, datetime.datetime]], datetime.datetime, float, NoneType]) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
name
relative_to(ts: Union[faust.types.tables.FieldDescriptorT, typing.Callable[[typing.Union[faust.types.events.EventT, NoneType]], typing.Union[float, datetime.datetime]], datetime.datetime, float, NoneType]) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_now() → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_field(field: faust.types.models.FieldDescriptorT) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_stream() → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
get_timestamp(event: faust.types.events.EventT = None) → float[source]
Return type:float
on_recover(fun: Callable[Awaitable[NoneType]]) → Callable[Awaitable[NoneType]][source]
Return type:Callable[[], Awaitable[None]]
get_relative_timestamp

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: Union[str, 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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None

Base Consumer.

ack(message: faust.types.tuples.Message) → bool
Return type:bool
close() → None
Return type:None
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool

Maybe commit the offset for all or specific topics.

Parameters:topics (Optional[AbstractSet[Union[str, TP]]]) – Set containing topics and/or TopicPartitions to commit.
Return type:bool
consumer_stopped_errors = ()
coroutine force_commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool
Return type:bool
logger = <Logger faust.transport.consumer (WARNING)>
coroutine maybe_wait_for_commit_to_finish(self) → bool
Return type:bool
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
Return type:None
coroutine on_stop(self) → None

Called every time before the service is stopped/restarted.

Return type:None
coroutine on_task_error(self, exc: BaseException) → None
Return type:None
track_message(message: faust.types.tuples.Message) → None
Return type:None
unacked
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 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
key_partition(topic: str, key: bytes) → faust.types.tuples.TP
Return type:TP
logger = <Logger faust.transport.producer (WARNING)>
coroutine send(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata
Return type:RecordMetadata
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.
acks_enabled_for(topic: str) → bool
Return type:bool
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:bool
discard(topic: Any) → None

Remove an element. Do not raise an exception if absent.

Return type:None
label

Label used for graphs. :rtype: str

logger = <Logger faust.transport.conductor (WARNING)>
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
Return type:None
shortlabel

Label used for logging. :rtype: str

coroutine wait_for_subscriptions(self) → None
Return type:None
class Fetcher(app: faust.types.app.AppT, **kwargs) → None
logger = <Logger faust.transport.consumer (WARNING)>
coroutine on_stop(self) → None

Called every time before the service is stopped/restarted.

Return type:None
create_consumer(callback: Callable[faust.types.tuples.Message, Awaitable], **kwargs) → faust.types.transports.ConsumerT[source]
Return type:ConsumerT[]
create_producer(**kwargs) → faust.types.transports.ProducerT[source]
Return type:ProducerT[]
create_conductor(**kwargs) → faust.types.transports.ConductorT[source]
Return type:ConductorT[]
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)>
acks_enabled_for(topic: str) → bool[source]
Return type:bool
clear() → None[source]

This is slow (creates N new iterators!) but effective.

Return type:None
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]
Return type:bool
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 wait_for_subscriptions(self) → None[source]
Return type:None
add(topic: Any) → None[source]

Add an element.

Return type:None
discard(topic: Any) → None[source]

Remove an element. Do not raise an exception if absent.

Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, 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.

track_message(message: faust.types.tuples.Message) → None[source]
Return type:None
ack(message: faust.types.tuples.Message) → bool[source]
Return type:bool
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool[source]

Maybe commit the offset for all or specific topics.

Parameters:topics (Optional[AbstractSet[Union[str, TP]]]) – Set containing topics and/or TopicPartitions to commit.
Return type:bool
coroutine force_commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool[source]
Return type:bool
coroutine maybe_wait_for_commit_to_finish(self) → bool[source]
Return type:bool
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 on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine on_task_error(self, exc: BaseException) → None[source]
Return type:None
coroutine wait_empty(self) → None[source]

Wait for all messages that started processing to be acked.

Return type:None
close() → None[source]
Return type:None
unacked
class faust.transport.base.Fetcher(app: faust.types.app.AppT, **kwargs) → None[source]
logger = <Logger faust.transport.consumer (WARNING)>
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
class faust.transport.base.Producer(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]

Base Producer.

key_partition(topic: str, key: bytes) → faust.types.tuples.TP[source]
Return type:TP
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: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
faust.transport.conductor

The conductor delegates messages from the consumer to the streams.

class faust.transport.conductor.ConductorCompiler[source]
build(conductor: faust.transport.conductor.Conductor, tp: faust.types.tuples.TP, channels: MutableSet[faust.transport.conductor.Topic]) → Callable[faust.types.tuples.Message, Awaitable][source]
Return type:Callable[[Message], Awaitable[+T_co]]
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)>
acks_enabled_for(topic: str) → bool[source]
Return type:bool
clear() → None[source]

This is slow (creates N new iterators!) but effective.

Return type:None
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]
Return type:bool
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 wait_for_subscriptions(self) → None[source]
Return type:None
add(topic: Any) → None[source]

Add an element.

Return type:None
discard(topic: Any) → None[source]

Remove an element. Do not raise an exception if absent.

Return type:None
label

Label used for graphs. :rtype: str

shortlabel

Label used for logging. :rtype: str

faust.transport.consumer

Consumer

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 it’s 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 tp+offset combination as “commitable”
    • 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 comitted 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]
logger = <Logger faust.transport.consumer (WARNING)>
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, 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.

track_message(message: faust.types.tuples.Message) → None[source]
Return type:None
ack(message: faust.types.tuples.Message) → bool[source]
Return type:bool
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool[source]

Maybe commit the offset for all or specific topics.

Parameters:topics (Optional[AbstractSet[Union[str, TP]]]) – Set containing topics and/or TopicPartitions to commit.
Return type:bool
coroutine force_commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool[source]
Return type:bool
coroutine maybe_wait_for_commit_to_finish(self) → bool[source]
Return type:bool
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 on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine on_task_error(self, exc: BaseException) → None[source]
Return type:None
coroutine wait_empty(self) → None[source]

Wait for all messages that started processing to be acked.

Return type:None
close() → None[source]
Return type:None
unacked
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.

key_partition(topic: str, key: bytes) → faust.types.tuples.TP[source]
Return type:TP
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: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
faust.transport.drivers

Transport registry.

faust.transport.drivers.aiokafka

Message transport using aiokafka.

class faust.transport.drivers.aiokafka.Consumer(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None[source]

Kafka consumer using aiokafka.

logger = <Logger faust.transport.drivers.aiokafka (WARNING)>
RebalanceListener

alias of ConsumerRebalanceListener

fetch_timeout = 10.0
consumer_stopped_errors = (<class 'aiokafka.errors.ConsumerStoppedError'>,)
on_init() → None[source]
Return type:None
assignment() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
highwater(tp: faust.types.tuples.TP) → int[source]
Return type:int
close() → 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] = 30.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:MutableMapping[TP, int]
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:MutableMapping[TP, int]
coroutine on_restart(self) → None[source]

Called every time when the service is restarted.

Return type:None
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine pause_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine perform_seek(self) → None[source]
Return type:None
coroutine position(self, tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
coroutine resume_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine seek(self, partition: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
coroutine subscribe(self, topics: Iterable[str]) → 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)>
on_init() → None[source]
Return type:None
key_partition(topic: str, key: bytes) → faust.types.tuples.TP[source]
Return type:TP
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 on_restart(self) → None[source]

Called every time when the service is restarted.

Return type:None
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None[source]

Called every time before the service is stopped/restarted.

Return type:None
coroutine send(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
class faust.transport.drivers.aiokafka.Transport(*args, **kwargs) → None[source]

Kafka transport using aiokafka.

class Consumer(transport: faust.types.transports.TransportT, callback: Callable[faust.types.tuples.Message, Awaitable], on_partitions_revoked: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, commit_interval: float = None, commit_livelock_soft_timeout: float = None, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None

Kafka consumer using aiokafka.

RebalanceListener

alias of ConsumerRebalanceListener

assignment() → Set[faust.types.tuples.TP]
Return type:Set[TP]
close() → None
Return type:None
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
coroutine earliest_offsets(self, *partitions) → MutableMapping[faust.types.tuples.TP, int]
Return type:MutableMapping[TP, int]
fetch_timeout = 10.0
getmany(timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]]
Return type:AsyncIterator[Tuple[TP, Message]]
highwater(tp: faust.types.tuples.TP) → int
Return type:int
coroutine highwaters(self, *partitions) → MutableMapping[faust.types.tuples.TP, int]
Return type:MutableMapping[TP, int]
logger = <Logger faust.transport.drivers.aiokafka (WARNING)>
on_init() → None
Return type:None
coroutine on_restart(self) → None

Called every time when the service is restarted.

Return type:None
coroutine on_start(self) → None

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None

Called every time before the service is stopped/restarted.

Return type:None
coroutine pause_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None
Return type:None
coroutine perform_seek(self) → None
Return type:None
coroutine position(self, tp: faust.types.tuples.TP) → Union[int, NoneType]
Return type:Optional[int]
coroutine resume_partitions(self, 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 subscribe(self, topics: Iterable[str]) → None
Return type:None
class Producer(transport: faust.types.transports.TransportT, loop: asyncio.events.AbstractEventLoop = None, **kwargs) → None

Kafka producer using aiokafka.

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
key_partition(topic: str, key: bytes) → faust.types.tuples.TP
Return type:TP
logger = <Logger faust.transport.drivers.aiokafka (WARNING)>
on_init() → None
Return type:None
coroutine on_restart(self) → None

Called every time when the service is restarted.

Return type:None
coroutine on_start(self) → None

Called every time before the service is started/restarted.

Return type:None
coroutine on_stop(self) → None

Called every time before the service is stopped/restarted.

Return type:None
coroutine send(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata
Return type:RecordMetadata
default_port = 9092
driver_version = 'aiokafka=0.4.18'
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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, 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 = ()
assignment() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
highwater(tp: faust.types.tuples.TP) → int[source]
Return type:int
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:MutableMapping[TP, int]
getmany(*partitions, 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:MutableMapping[TP, int]
coroutine pause_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine perform_seek(self) → None[source]
Return type:None
coroutine position(self, tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
coroutine resume_partitions(self, partitions: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine seek(self, partition: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
coroutine seek_to_beginning(self, *partitions) → None[source]
Return type:None
coroutine seek_to_latest(self, *partitions) → None[source]
Return type:None
coroutine subscribe(self, topics: Iterable[str]) → None[source]
Return type:None
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: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, 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.

assignment() → Set[faust.types.tuples.TP]
Return type:Set[TP]
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:MutableMapping[TP, int]
getmany(*partitions, timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]]
Return type:AsyncIterator[Tuple[TP, Message]]
highwater(tp: faust.types.tuples.TP) → int
Return type:int
coroutine highwaters(self, *partitions) → MutableMapping[faust.types.tuples.TP, int]
Return type:MutableMapping[TP, int]
coroutine pause_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None
Return type:None
coroutine perform_seek(self) → None
Return type:None
coroutine position(self, tp: faust.types.tuples.TP) → Union[int, NoneType]
Return type:Optional[int]
coroutine resume_partitions(self, 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 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: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata
Return type:RecordMetadata
default_port = 9092
driver_version = 'memory-1.0.30'
coroutine send(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
coroutine subscribe(self, topics: Iterable[str]) → None[source]
Return type:None

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.

validate() → None[source]
Return type:None
num_assigned(active: bool) → int[source]
Return type:int
get_unassigned(num_partitions: int, active: bool) → Set[int][source]
Return type:Set[int]
pop_partition(active: bool) → int[source]
Return type:int
unassign_partition(partition: int, active: bool) → None[source]
Return type:None
assign_partition(partition: int, active: bool) → None[source]
Return type:None
unassign_extras(capacity: int, replicas: int) → None[source]
Return type:None
partition_assigned(partition: int, active: bool) → bool[source]
Return type:bool
promote_standby_to_active(standby_partition: int) → None[source]
Return type:None
get_assigned_partitions(active: bool) → Set[int][source]
Return type:Set[int]
can_assign(partition: int, active: bool) → bool[source]
Return type:bool
class faust.assignor.client_assignment.ClientAssignment(actives, standbys, *, __strict__=True, __faust=None, **kwargs) → NoneType[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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
active_tps
standby_tps
kafka_protocol_assignment(table_manager: faust.types.tables.TableManagerT) → Sequence[Tuple[str, List[int]]][source]
Return type:Sequence[Tuple[str, List[int]]]
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:CopartitionedAssignment
asdict()
class faust.assignor.client_assignment.ClientMetadata(assignment, url, changelog_distribution, *, __strict__=True, __faust=None, **kwargs) → NoneType[source]

Client Metadata data model.

asdict()
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
faust.assignor.cluster_assignment

Cluster assignement.

faust.assignor.cluster_assignment.CopartMapping

alias of typing.MutableMapping

class faust.assignor.cluster_assignment.ClusterAssignment(subscriptions=None, assignments=None, *, __strict__=True, __faust=None, **kwargs) → NoneType[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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
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>
Parameters:
  • field (str) – Name of field.
  • type (Type) – Field value type.
  • model (Type) – Model class the field belongs to.
  • required (bool) – Set to false if field is optional.
  • default (Any) – Default value when required=False.
topics() → Set[str][source]
Return type:Set[str]
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:MutableMapping[str, CopartitionedAssignment]
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.

get_assignment() → MutableMapping[str, faust.assignor.client_assignment.CopartitionedAssignment][source]
Return type:MutableMapping[str, CopartitionedAssignment]
faust.assignor.leader_assignor

Leader assignor.

class faust.assignor.leader_assignor.LeaderAssignor(app: faust.types.app.AppT, **kwargs) → None[source]

Leader assignor, ensures election of a leader.

is_leader() → bool[source]
Return type:bool
logger = <Logger faust.assignor.leader_assignor (WARNING)>
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
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 https://github.com/dpkp/kafka-python/blob/master/ kafka/coordinator/assignors/abstract.py

changelog_distribution
on_assignment(assignment: rhkafka.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]) → rhkafka.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: rhkafka.cluster.ClusterMetadata, member_metadata: MutableMapping[str, rhkafka.coordinator.protocol.ConsumerProtocolMemberMetadata]) → MutableMapping[str, rhkafka.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:

dict

name

.name should be a string identifying the assignor :rtype: str

version
assigned_standbys() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
assigned_actives() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
table_metadata(topic: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
tables_metadata() → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
key_store(topic: str, key: bytes) → yarl.URL[source]
Return type:URL
is_active(tp: faust.types.tuples.TP) → bool[source]
Return type:bool
is_standby(tp: faust.types.tuples.TP) → bool[source]
Return type:bool

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]]]

A sink can be – Agent, Channel or callable/async callable taking value as argument.

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.

cancel() → None[source]
Return type:None
coroutine on_isolated_partition_assigned(self, tp: faust.types.tuples.TP) → None[source]
Return type:None
coroutine on_isolated_partition_revoked(self, tp: faust.types.tuples.TP) → None[source]
Return type:None
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[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], *, name: str = None, app: faust.types.agents.AppT = None, channel: Union[str, faust.types.channels.ChannelT] = None, concurrency: int = 1, sink: Iterable[Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, on_error: Callable[[_ForwardRef('AgentT'), BaseException], Awaitable] = None, supervisor_strategy: Type[mode.types.supervisors.SupervisorStrategyT] = None, help: str = None, key_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.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:AgentTestWrapperT[]
add_sink(sink: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]) → None[source]
Return type:None
stream(**kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
info() → Mapping[source]
Return type:Mapping[~KT, +VT_co]
clone(*, cls: Type[_ForwardRef('AgentT')] = None, **kwargs) → faust.types.agents.AgentT[source]
Return type:AgentT[]
get_topic_names() → Iterable[str][source]
Return type:Iterable[str]
channel
channel_iterator
coroutine ask(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Any[source]
Return type:Any
coroutine cast(self, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, *, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None) → None[source]
Return type:None
coroutine join(self, values: Union[typing.AsyncIterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]], typing.Iterable[typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvjoin(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → List[Any][source]
Return type:List[Any]
coroutine kvmap(self, items: Union[typing.AsyncIterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]], typing.Iterable[typing.Tuple[typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], typing.Union[bytes, faust.types.core.ModelT, typing.Any]]]], reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None) → AsyncIterator[str][source]
coroutine map(self, values: Union[typing.AsyncIterable, typing.Iterable], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, reply_to: Union[_ForwardRef('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, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, *, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
class faust.types.agents.AgentManagerT(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[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
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, typing.Any] = None, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, *, reply_to: Union[_ForwardRef('AgentT'), faust.types.channels.ChannelT, str] = None, correlation_id: str = None, wait: bool = True) → faust.types.events.EventT[source]
Return type:EventT[]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
to_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], *, partition: int = 0, offset: int = 0, timestamp: float = None, timestamp_type: int = 0) → faust.types.tuples.Message[source]
Return type:Message
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

faust.App.

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.

unassigned = False
on_configured(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.>
on_before_configured(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.>
on_after_configured(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.>
on_partitions_assigned(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.>
on_partitions_revoked(sender: T_contra = None, *args, **kwargs) → None = <Signal: AppT.>
on_worker_init(sender: T_contra = None, *args, **kwargs) → None = <SyncSignal: AppT.>
config_from_object(obj: Any, *, silent: bool = False, force: bool = False) → None[source]
Return type:None
finalize() → None[source]
Return type:None
main() → None[source]
Return type:None
worker_init() → None[source]
Return type:None
discover(*extra_modules, categories: Iterable[str] = ('a', 'b', 'c'), ignore: Iterable[str] = ('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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, 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 = 1, 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[_ForwardRef('AgentT'), faust.types.channels.ChannelT, typing.Callable[[typing.Any], typing.Union[typing.Awaitable, NoneType]]]] = None, isolated_partitions: bool = False, **kwargs) → Callable[Callable[Union[typing.AsyncIterator, faust.types.streams.StreamT], Union[typing.Awaitable, typing.AsyncIterable]], faust.types.agents.AgentT][source]
Return type:Callable[[Callable[[Union[AsyncIterator[+T_co], StreamT[+T_co]]], Union[Awaitable[+T_co], AsyncIterable[+T_co]]]], AgentT[]]
task(fun: Union[typing.Callable[[_ForwardRef('AppT')], typing.Awaitable], typing.Callable[[], typing.Awaitable]]) → Callable[source]
Return type:Callable
timer(interval: Union[datetime.timedelta, float, str], on_leader: bool = False) → Callable[source]
Return type:Callable
service(cls: Type[mode.types.services.ServiceT]) → Type[mode.types.services.ServiceT][source]
Return type:Type[ServiceT[]]
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[]
page(path: str, *, base: Type[faust.types.web.View] = <class 'faust.types.web.View'>) → Callable[Union[typing.Type[faust.types.web.View], typing.Callable[[faust.types.web.View, faust.types.web.Request], typing.Awaitable[faust.types.web.Response]]], Type[faust.types.web.Site]][source]
Return type:Callable[[Union[Type[View], Callable[[View, Request], Awaitable[Response]]]], Type[Site]]
table_route(table: faust.types.tables.CollectionT, shard_param: str) → Callable[Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]], Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]]][source]
Return type:Callable[[Callable[[View, Request], Awaitable[Response]]], Callable[[View, Request], Awaitable[Response]]]
command(*options, base: Type[faust.types.app.AppCommand] = None, **kwargs) → Callable[Callable, Type[faust.types.app.AppCommand]][source]
Return type:Callable[[Callable], Type[AppCommand]]
is_leader() → bool[source]
Return type:bool
FlowControlQueue(maxsize: int = None, *, clear_on_resume: bool = False, loop: asyncio.events.AbstractEventLoop = None) → mode.utils.queues.ThrowableQueue[source]
Return type:ThrowableQueue
Worker(**kwargs) → faust.types.app.WorkerT[source]
Return type:WorkerT
on_webserver_init(web: faust.types.web.Web) → None[source]
Return type:None
conf
transport
producer
consumer
tables[source]
topics[source]
monitor
flow_control[source]
http_client
assignor
coroutine maybe_start_client(self) → None[source]
Return type:None
maybe_start_producer[source]
router
coroutine send(self, channel: Union[faust.types.channels.ChannelT, str], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine start_client(self) → None[source]
Return type:None
serializers
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]
assigned_standbys() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
assigned_actives() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
is_active(tp: faust.types.tuples.TP) → bool[source]
Return type:bool
is_standby(tp: faust.types.tuples.TP) → bool[source]
Return type:bool
key_store(topic: str, key: bytes) → yarl.URL[source]
Return type:URL
table_metadata(topic: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
tables_metadata() → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
class faust.types.assignor.LeaderAssignorT(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]
is_leader() → bool[source]
Return type:bool
faust.types.channels
class faust.types.channels.AppT[source]
class faust.types.channels.EventT[source]
class faust.types.channels.ModelArg[source]
class faust.types.channels.StreamT[source]
class faust.types.channels.ConsumerT[source]
class faust.types.channels.TPorTopicSet[source]
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: Union[faust.types.channels.ChannelT, NoneType] = 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[]
stream(**kwargs) → faust.types.channels.StreamT[source]
Return type:StreamT
get_topic_name() → str[source]
Return type:str
as_future_message(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None) → faust.types.tuples.FutureMessage[source]
Return type:FutureMessage[]
prepare_key(key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], key_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
prepare_value(value: Union[bytes, faust.types.core.ModelT, typing.Any], value_serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]
Return type:Any
empty() → bool[source]
Return type:bool
on_stop_iteration() → None[source]
Return type:None
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
subscriber_count
queue
coroutine declare(self) → None[source]
Return type:None
coroutine decode(self, message: faust.types.tuples.Message, *, propagate: bool = False) → faust.types.channels.EventT[source]
Return type:EventT
coroutine deliver(self, message: faust.types.tuples.Message) → None[source]
Return type:None
coroutine get(self, *, timeout: Union[datetime.timedelta, float, str] = None) → Any[source]
Return type:Any
maybe_declare[source]
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:Awaitable[RecordMetadata]
coroutine put(self, value: Any) → None[source]
Return type:None
coroutine send(self, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
faust.types.codecs
class faust.types.codecs.CodecT(children: Tuple[_ForwardRef('CodecT'), ...] = None, **kwargs)[source]

Abstract type for an encoder/decoder.

dumps(obj: Any) → bytes[source]
Return type:bytes
loads(s: bytes) → Any[source]
Return type:Any
clone(*children) → faust.types.codecs.CodecT[source]
Return type:CodecT
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.events
class faust.types.events.AppT[source]
class faust.types.events.ChannelT[source]
class faust.types.events.EventT(app: faust.types.events.AppT, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], message: faust.types.tuples.Message) → None[source]
app
key
value
message
acked
ack() → bool[source]
Return type:bool
coroutine forward(self, channel: Union[str, faust.types.events.ChannelT], key: Any = None, value: Any = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send(self, channel: Union[str, faust.types.events.ChannelT], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType] = None, value: Union[bytes, faust.types.core.ModelT, typing.Any] = None, partition: int = None, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, callback: Callable[faust.types.tuples.FutureMessage, Union[NoneType, typing.Awaitable[NoneType]]] = None, force: bool = False) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
faust.types.fixups
class faust.types.fixups.FixupT(app: faust.types.fixups.AppT) → None[source]
enabled() → bool[source]
Return type:bool
autodiscover_modules() → Iterable[str][source]
Return type:Iterable[str]
on_worker_init() → None[source]
Return type:None
faust.types.joins
class faust.types.joins.JoinT(*, stream: faust.types.streams.JoinableT, fields: Tuple[faust.types.models.FieldDescriptorT, ...]) → None[source]
coroutine process(self, event: faust.types.events.EventT) → Union[faust.types.events.EventT, NoneType][source]
Return type:Optional[EventT[]]
faust.types.models
class faust.types.models.Converter(*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
clone_defaults() → faust.types.models.ModelOptions[source]
Return type:ModelOptions
fields = None

Index – Flattened view of __annotations__ in MRO order.

fieldset = None

Index – Set of required field names, for fast argument checking.

fieldpos = None

Index – Positional argument index to field name. Used by Record.__init__ to map positional arguments to fields.

optionalset = None

Index – Set of optional field names, for fast argument checking.

models = None

Index – Mapping of fields that are ModelT

modelattrs = None
converse = None

Index – Mapping of fields that are not builtin-types. E.g. datetime.

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[_ForwardRef('ModelT')] = None) → faust.types.models.ModelT[source]
Return type:ModelT
classmethod loads(s: bytes, *, default_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → faust.types.models.ModelT[source]
Return type:ModelT
dumps(*, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → bytes[source]
Return type:bytes
derive(*objects, **fields) → faust.types.models.ModelT[source]
Return type:ModelT
to_representation() → Any[source]
Return type:Any
class faust.types.models.FieldDescriptorT(field: str, type: Type, model: Type[faust.types.models.ModelT], required: bool = True, default: Any = None, parent: Union[faust.types.models.FieldDescriptorT, NoneType] = None) → None[source]
required = True
default = None
getattr(obj: faust.types.models.ModelT) → Any[source]
Return type:Any
ident
faust.types.router

Types for module faust.router.

class faust.types.router.AppT[source]
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, typing.Any, NoneType]) → yarl.URL[source]
Return type:URL
table_metadata(table_name: str) → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
tables_metadata() → MutableMapping[str, MutableMapping[str, List[int]]][source]
Return type:MutableMapping[str, MutableMapping[str, List[int]]]
coroutine route_req(self, table_name: str, key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], web: faust.types.web.Web, request: faust.types.web.Request) → faust.types.web.Response[source]
Return type:Response
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) → None[source]
Return type:None
on_stream_event_out(tp: faust.types.tuples.TP, offset: int, stream: faust.types.streams.StreamT, event: faust.types.events.EventT) → None[source]
Return type:None
on_topic_buffer_full(topic: faust.types.topics.TopicT) → 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_get(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_table_set(table: faust.types.tables.CollectionT, key: Any, value: Any) → None[source]
Return type:None
on_table_del(table: faust.types.tables.CollectionT, key: Any) → None[source]
Return type:None
on_commit_initiated(consumer: faust.types.transports.ConsumerT) → Any[source]
Return type:Any
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, keysize: int, valsize: int) → Any[source]
Return type:Any
on_send_completed(producer: faust.types.transports.ProducerT, state: Any) → None[source]
Return type:None
class faust.types.sensors.SensorT(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]
class faust.types.sensors.SensorDelegateT[source]
add(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
remove(sensor: faust.types.sensors.SensorT) → None[source]
Return type:None
faust.types.serializers
class faust.types.serializers.RegistryT(key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 'json') → None[source]
loads_key(typ: Union[faust.types.serializers.ModelArg, NoneType], key: Union[bytes, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Union[bytes, faust.types.core.ModelT, typing.Any, NoneType][source]
Return type:Union[bytes, ModelT, Any, None]
loads_value(typ: Union[faust.types.serializers.ModelArg, NoneType], value: Union[bytes, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Any[source]
Return type:Any
dumps_key(typ: Union[faust.types.serializers.ModelArg, NoneType], key: Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Union[bytes, NoneType][source]
Return type:Optional[bytes]
dumps_value(typ: Union[faust.types.serializers.ModelArg, NoneType], value: Union[bytes, faust.types.core.ModelT, typing.Any], *, serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None) → Union[bytes, NoneType][source]
Return type:Optional[bytes]
faust.types.settings
class faust.types.settings.Settings(id: str, *, version: int = None, broker: Union[str, yarl.URL] = None, broker_client_id: str = 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, agent_supervisor: Union[typing.Type[mode.types.supervisors.SupervisorStrategyT], str] = None, store: Union[str, yarl.URL] = None, autodiscover: Union[bool, typing.Iterable[str], typing.Callable[[], typing.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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, loghandlers: List[logging.StreamHandler] = None, table_cleanup_interval: Union[datetime.timedelta, float, str] = None, table_standby_replicas: int = None, topic_replication_factor: int = None, topic_partitions: int = 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, 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, 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, worker_redirect_stdouts: bool = None, worker_redirect_stdouts_level: Union[int, str] = None, Agent: Union[typing.Type[faust.types.agents.AgentT], str] = None, Stream: Union[typing.Type[faust.types.streams.StreamT], str] = None, Table: Union[typing.Type[faust.types.tables.TableT], str] = None, TableManager: Union[typing.Type[faust.types.tables.TableManagerT], str] = None, Serializers: Union[typing.Type[faust.types.serializers.RegistryT], str] = None, Worker: Union[typing.Type[faust.types.settings.WorkerT], str] = None, PartitionAssignor: Union[typing.Type[faust.types.assignor.PartitionAssignorT], str] = None, LeaderAssignor: Union[typing.Type[faust.types.assignor.LeaderAssignorT], str] = None, Router: Union[typing.Type[faust.types.router.RouterT], str] = None, Topic: Union[typing.Type[faust.types.topics.TopicT], str] = None, HttpClient: Union[typing.Type[faust.types.web.HttpClientT], str] = None, Monitor: Union[typing.Type[faust.types.sensors.SensorT], str] = None, url: Union[str, yarl.URL] = None, **kwargs) → None[source]
classmethod setting_names() → Set[str][source]
Return type:Set[str]
id_format = '{id}-v{self.version}'
origin = None
autodiscover = False
broker_client_id = 'faust-1.0.30'
broker_commit_every = 10000
broker_check_crcs = True
key_serializer = 'json'
value_serializer = 'json'
table_standby_replicas = 1
topic_replication_factor = 1
topic_partitions = 8
reply_create_topic = False
stream_buffer_maxsize = 4096
stream_wait_empty = False
stream_ack_cancelled_tasks = False
stream_ack_exceptions = True
stream_publish_on_commit = True
producer_linger_ms = 0
producer_max_batch_size = 16384
producer_acks = -1
producer_max_request_size = 1000000
producer_compression_type = None
worker_redirect_stdouts = True
worker_redirect_stdouts_level = 'WARN'
reply_to_prefix = 'f-reply-'
prepare_id(id: str) → str[source]
Return type:str
prepare_datadir(datadir: Union[str, pathlib.Path]) → pathlib.Path[source]
Return type:Path
prepare_tabledir(tabledir: Union[str, pathlib.Path]) → pathlib.Path[source]
Return type:Path
name
id
version
broker
store
canonical_url
datadir
appdir
tabledir
broker_session_timeout
broker_heartbeat_interval
broker_commit_interval
broker_commit_livelock_soft_timeout
table_cleanup_interval
reply_expires
agent_supervisor
Agent
Stream
Table
TableManager
Serializers
Worker
PartitionAssignor
LeaderAssignor
Router
Topic
HttpClient
Monitor
faust.types.stores
class faust.types.stores.StoreT(url: Union[str, yarl.URL], app: faust.types.stores.AppT, *, 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, NoneType] = '', value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = '', **kwargs) → None[source]
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
set_persisted_offset(tp: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
apply_changelog_batch(batch: Iterable[faust.types.events.EventT], to_key: Callable[Any, Any], to_value: Callable[Any, Any]) → None[source]
Return type:None
reset_state() → None[source]
Return type:None
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_partitions_assigned(self, table: faust.types.stores.CollectionT, assigned: Set[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine on_partitions_revoked(self, table: faust.types.stores.CollectionT, revoked: Set[faust.types.tuples.TP]) → 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: Union[faust.types.streams.StreamT, NoneType] = None, active_partitions: Set[faust.types.tuples.TP] = None, enable_acks: bool = True, 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
get_active_stream() → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
add_processor(processor: Callable[T]) → None[source]
Return type:None
info() → Mapping[str, Any][source]
Return type:Mapping[str, Any]
clone(**kwargs) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
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]
echo(*channels) → faust.types.streams.StreamT[source]
Return type:StreamT[+T_co]
group_by(key: Union[faust.types.models.FieldDescriptorT, typing.Callable[[~T], typing.Union[bytes, faust.types.core.ModelT, typing.Any, NoneType]]], *, 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[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, value_type: Union[typing.Type[faust.types.models.ModelT], typing.Type[bytes], typing.Type[str]] = None, prefix: str = '', suffix: str = '') → faust.types.topics.TopicT[source]
Return type:TopicT[]
coroutine ack(self, event: faust.types.events.EventT) → bool[source]
Return type:bool
coroutine events(self) → AsyncIterable[faust.types.events.EventT][source]
coroutine items(self) → AsyncIterator[Tuple[Union[bytes, faust.types.core.ModelT, typing.Any, NoneType], T_co]][source]
coroutine send(self, value: T_contra) → None[source]
Return type:None
coroutine take(self, max_: int, within: Union[datetime.timedelta, float, str]) → AsyncIterable[Sequence[T_co]][source]
coroutine throw(self, exc: BaseException) → None[source]
Return type:None
faust.types.tables
faust.types.tables.RecoverCallback

alias of typing.Callable

faust.types.tables.ChangelogEventCallback

alias of typing.Callable

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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = None, recovery_buffer_size: int = 1000, standby_buffer_size: int = None, extra_topic_configs: Mapping[str, Any] = None, **kwargs) → None[source]
StateStore = None
changelog_topic
apply_changelog_batch(batch: Iterable[faust.types.events.EventT]) → None[source]
Return type:None
persisted_offset(tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
reset_state() → None[source]
Return type:None
on_recover(fun: Callable[Awaitable[NoneType]]) → Callable[Awaitable[NoneType]][source]
Return type:Callable[[], Awaitable[None]]
coroutine call_recover_callbacks(self) → None[source]
Return type:None
coroutine need_active_standby_for(self, tp: faust.types.tuples.TP) → bool[source]
Return type:bool
coroutine on_changelog_event(self, event: faust.types.events.EventT) → 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
faust.types.tables.CollectionTps

alias of typing.MutableMapping

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[NoneType]] = None, on_changelog_event: Callable[faust.types.events.EventT, Awaitable[NoneType]] = 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) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
hopping(size: Union[datetime.timedelta, float, str], step: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
tumbling(size: Union[datetime.timedelta, float, str], expires: Union[datetime.timedelta, float, str] = None) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
as_ansitable(*, key: str = 'Key', value: str = 'Value', sort: bool = False, sortkey: Callable[Any, Any] = <function TableT.<lambda>>, title: str = 'Title') → str[source]
Return type:str
class faust.types.tables.TableManagerT(app: faust.types.tables.AppT, **kwargs) → None[source]
add(table: faust.types.tables.CollectionT) → faust.types.tables.CollectionT[source]
Return type:CollectionT[]
changelog_topics
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
class faust.types.tables.ChangelogReaderT(*, beacon: mode.utils.types.trees.NodeT = None, loop: asyncio.events.AbstractEventLoop = None) → None[source]
class faust.types.tables.WindowSetT(key: Any, table: faust.types.tables.TableT, wrapper: faust.types.tables.WindowWrapperT, event: faust.types.events.EventT = None) → None[source]
apply(op: Callable[[Any, Any], Any], value: Any, event: faust.types.events.EventT = None) → faust.types.tables.WindowSetT[source]
Return type:WindowSetT[]
value(event: faust.types.events.EventT = None) → Any[source]
Return type:Any
current(event: faust.types.events.EventT = None) → Any[source]
Return type:Any
now() → Any[source]
Return type:Any
delta(d: Union[datetime.timedelta, float, str], event: faust.types.events.EventT = None) → Any[source]
Return type:Any
class faust.types.tables.WindowWrapperT(table: faust.types.tables.TableT, *, relative_to: Union[faust.types.tables.FieldDescriptorT, typing.Callable[[typing.Union[faust.types.events.EventT, NoneType]], typing.Union[float, datetime.datetime]], datetime.datetime, float, NoneType] = None) → None[source]
name
clone(relative_to: Union[faust.types.tables.FieldDescriptorT, typing.Callable[[typing.Union[faust.types.events.EventT, NoneType]], typing.Union[float, datetime.datetime]], datetime.datetime, float, NoneType]) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_now() → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_field(field: faust.types.tables.FieldDescriptorT) → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
relative_to_stream() → faust.types.tables.WindowWrapperT[source]
Return type:WindowWrapperT[]
get_timestamp(event: faust.types.events.EventT = None) → float[source]
Return type:float
get_relative_timestamp
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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]
topics = None

Iterable/Sequence of topic names to subscribe to.

retention = None

Topic retention setting – expiry time in seconds for messages in the topic.

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

Mark topic as internal – it’s owned by us and we are allowed to create or delete the topic as necessary.

pattern

or instead of topics, a regular expression used to match topics we want to subscribe to. :rtype: Optional[Pattern[AnyStr]]

partitions
derive(**kwargs) → faust.types.channels.ChannelT[source]
Return type:ChannelT[]
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

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[NoneType]], on_partitions_assigned: Callable[Set[faust.types.tuples.TP], Awaitable[NoneType]], *, 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.

track_message(message: faust.types.tuples.Message) → None[source]
Return type:None
ack(message: faust.types.tuples.Message) → bool[source]
Return type:bool
assignment() → Set[faust.types.tuples.TP][source]
Return type:Set[TP]
highwater(tp: faust.types.tuples.TP) → int[source]
Return type:int
close() → None[source]
Return type:None
unacked
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]] = None) → bool[source]
Return type:bool
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:MutableMapping[TP, int]
coroutine getmany(self, timeout: float) → AsyncIterator[Tuple[faust.types.tuples.TP, faust.types.tuples.Message]][source]
coroutine highwaters(self, *partitions) → MutableMapping[faust.types.tuples.TP, int][source]
Return type:MutableMapping[TP, int]
coroutine on_task_error(self, exc: BaseException) → None[source]
Return type:None
coroutine pause_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine perform_seek(self) → None[source]
Return type:None
coroutine position(self, tp: faust.types.tuples.TP) → Union[int, NoneType][source]
Return type:Optional[int]
coroutine resume_partitions(self, tps: Iterable[faust.types.tuples.TP]) → None[source]
Return type:None
coroutine seek(self, partition: faust.types.tuples.TP, offset: int) → None[source]
Return type:None
coroutine subscribe(self, topics: Iterable[str]) → None[source]
Return type:None
coroutine wait_empty(self) → None[source]
Return type:None
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.

key_partition(topic: str, key: bytes) → faust.types.tuples.TP[source]
Return type:TP
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: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → Awaitable[faust.types.tuples.RecordMetadata][source]
Return type:Awaitable[RecordMetadata]
coroutine send_and_wait(self, topic: str, key: Union[bytes, NoneType], value: Union[bytes, NoneType], partition: Union[int, NoneType]) → faust.types.tuples.RecordMetadata[source]
Return type:RecordMetadata
class faust.types.transports.ConductorT(app: faust.types.transports.AppT, **kwargs) → None[source]
acks_enabled_for(topic: str) → bool[source]
Return type:bool
coroutine commit(self, topics: AbstractSet[Union[str, faust.types.tuples.TP]]) → bool[source]
Return type:bool
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 wait_for_subscriptions(self) → None[source]
Return type:None
class faust.types.transports.TransportT(url: Union[str, 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.

Conductor = None

The Conductor class used to delegate messages from Consumer to streams.

Fetcher = None

The Fetcher service used for this type of transport.

app = None

The faust.App that created this transport.

url = None

The URL to use for this transport (e.g. kafka – //localhost).

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:ConsumerT[]
create_producer(**kwargs) → faust.types.transports.ProducerT[source]
Return type:ProducerT[]
create_conductor(**kwargs) → faust.types.transports.ConductorT[source]
Return type:ConductorT[]
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

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

key_serializer

Alias for field number 4

value_serializer

Alias for field number 5

callback

Alias for field number 6

topic

Alias for field number 7

offset

Alias for field number 8

tp
ack(consumer: faust.types.tuples.ConsumerT) → None[source]
Return type:None
class faust.types.tuples.FutureMessage(message: faust.types.tuples.PendingMessage) → None[source]
set_result(result: faust.types.tuples.RecordMetadata) → None[source]

Mark the future done and set its result.

If the future is already done when this method is called, raises InvalidStateError.

Return type:None
class faust.types.tuples.Message(topic: str, partition: int, offset: int, timestamp: float, timestamp_type: int, key: Union[bytes, NoneType], value: Union[bytes, NoneType], checksum: Union[bytes, NoneType], 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)[source]
topic
partition
offset
timestamp
timestamp_type
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:

 messsage.stream_meta[id(stream)] = {
     'time_in': float,
     'time_out': float,
     'time_total': float,
}
ack(consumer: faust.types.tuples.ConsumerT, n: int = 1) → bool[source]
Return type:bool
incref(n: int = 1) → None[source]
Return type:None
decref(n: int = 1) → int[source]
Return type:int
classmethod from_message(message: Any, tp: faust.types.tuples.TP) → faust.types.tuples.Message[source]
Return type:Message
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
class faust.types.web.Request[source]
class faust.types.web.Response[source]
class faust.types.web.Web[source]
class faust.types.web.Site[source]
class faust.types.web.View[source]
faust.types.web.ViewGetHandler

alias of typing.Callable

faust.types.web.RoutedViewGetHandler

alias of typing.Callable

class faust.types.web.HttpClientT(*, connector=None, loop=None, cookies=None, headers=None, skip_auto_headers=None, auth=None, json_serialize=<function dumps>, request_class=<class 'aiohttp.client_reqrep.ClientRequest'>, response_class=<class 'aiohttp.client_reqrep.ClientResponse'>, ws_response_class=<class 'aiohttp.client_ws.ClientWebSocketResponse'>, version=HttpVersion(major=1, minor=1), cookie_jar=None, connector_owner=True, raise_for_status=False, read_timeout=<object object>, conn_timeout=None, timeout=<object object>, auto_decompress=True, trust_env=False, trace_configs=None)[source]
faust.types.windows
class faust.types.windows.WindowRange(*args, **kwargs)[source]
start

Alias for field number 0

end

Alias for field number 1

classmethod from_start(start: float, size: float) → faust.types.windows.WindowRange[source]
Return type:WindowRange
class faust.types.windows.WindowT(*args, **kwargs)[source]
expires = None
tz = None
ranges(timestamp: float) → List[faust.types.windows.WindowRange][source]
Return type:List[WindowRange]
stale(timestamp: float, latest_timestamp: float) → bool[source]
Return type:bool
current(timestamp: float) → faust.types.windows.WindowRange[source]
Return type:WindowRange
delta(timestamp: float, d: Union[datetime.timedelta, float, str]) → faust.types.windows.WindowRange[source]
Return type:WindowRange

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]
Return type:Callable
faust.utils.codegen.Method(name: str, args: List[str], body: List[str], **kwargs) → Callable[source]
Return type:Callable
faust.utils.codegen.InitMethod(args: List[str], body: List[str], **kwargs) → Callable[NoneType][source]
Return type:Callable[[], None]
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

Return type:Iterator[Sequence[int]]
faust.utils.iso8601
faust.utils.iso8601.parse(datestring: str) → datetime.datetime[source]

Parse and convert ISO 8601 string into a datetime object.

Return type:datetime
faust.utils.json

JSON utilities.

faust.utils.json.str_to_decimal(s: str, maxlen: int = 1000) → Union[decimal.Decimal, NoneType][source]

Convert string to Decimal.

Parameters:
  • s (str) – Number to convert.
  • maxlen (int) – Max length of string. Default is 100.
Raises:

ValueError – if length exceeds maximum length, or if value is not a valid number (e.g. Inf, NaN or sNaN).

Return type:

Optional[Decimal]

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]
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 a TypeError).

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:Any
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:str
faust.utils.json.loads(s: str, **kwargs) → Any[source]

Deserialize json string. See json.loads().

Return type:Any
faust.utils.platforms

Platform/OS utilities.

faust.utils.platforms.max_open_files() → Union[int, NoneType][source]

Return max number of open files, or None.

Return type:Optional[int]
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], NoneType] = None, **kwargs) → None[source]
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 value None which means all Venusian callback categories. The default is None.

The onerror argument should either be None or a callback function which behaves the same way as the onerror 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, including ImportError. If you use a custom onerror callback, you can change this behavior.

Here’s an example onerror callback that ignores ImportError:

import sys
def onerror(name):
    if not issubclass(sys.exc_info()[0], ImportError):
        raise # reraise the last exception

The name passed to onerror is the module or package dotted name that could not be imported due to an exception.

New in version 1.0: the onerror callback

The 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, pass ignore=['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 the package you’ve passed is imported as my.package, and you pass ignore=['.mymodule'], the my.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 or False. 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 use ignore=[re.compile('tests$').search]. If the callable returns True (or anything else truthy), the object is ignored, if it returns False (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 cause my.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 word tests 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
update() → None[source]
Return type:None
stop() → None[source]
Return type:None
write(s: str) → None[source]
Return type:None
begin() → None[source]
Return type:None
finish() → None[source]
Return type:None
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.

emit(_record: logging.LogRecord) → None[source]

Do whatever it takes to actually log the specified logging record.

This version is intended to be implemented by subclasses and so raises a NotImplementedError.

Return type:None
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:bool
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:str
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
update() → None[source]
Return type:None
stop() → None[source]
Return type:None
write(s: str) → None[source]
Return type:None
begin() → None[source]
Return type:None
finish() → None[source]
Return type:None
class faust.utils.terminal.spinners.SpinnerHandler(spinner: faust.utils.terminal.spinners.Spinner, **kwargs) → None[source]

A logger handler that iterates our progress spinner for each log.

emit(_record: logging.LogRecord) → None[source]

Do whatever it takes to actually log the specified logging record.

This version is intended to be implemented by subclasses and so raises a NotImplementedError.

Return type:None
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:str
faust.utils.terminal.tables.Table

alias of terminaltables.base_table.BaseTable

Web

faust.web.apps.graph

Web endpoint showing graph of running mode services.

class faust.web.apps.graph.Graph(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

Render image from graph of running services.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
class faust.web.apps.graph.Site(app: faust.types.app.AppT) → None[source]

Graph views.

views = {'/': <class 'faust.web.apps.graph.Graph'>}
faust.web.apps.router

HTTP endpoint showing partition routing destinations.

class faust.web.apps.router.TablesMetadata(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

List routes for all tables.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
class faust.web.apps.router.TableMetadata(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

List route for specific table.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
class faust.web.apps.router.KeyMetadata(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

List information about key.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
faust.web.apps.stats

HTTP endpoint showing statistics from the Faust monitor.

class faust.web.apps.stats.Stats(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

Monitor statistics.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
class faust.web.apps.stats.Assignment(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

Cluster assignment information.

coroutine get(self, request: faust.web.base.Request) → faust.web.base.Response[source]
Return type:Response
class faust.web.apps.stats.Site(app: faust.types.app.AppT) → None[source]

Statistics views.

views = {'/': <class 'faust.web.apps.stats.Stats'>, '/assignment/': <class 'faust.web.apps.stats.Assignment'>}
faust.web.base

Base interface for Web server and views.

class faust.web.base.Response[source]

Web server response and status.

class faust.web.base.Web(app: faust.types.app.AppT, *, port: int = None, bind: str = None, **kwargs) → None[source]

Web server and HTTP interface.

text(value: str, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
html(value: str, *, status: int = 200) → faust.web.base.Response[source]
Return type:Response
json(value: Any, *, status: int = 200) → faust.web.base.Response[source]
Return type:Response
bytes(value: bytes, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
route(pattern: str, handler: Callable) → None[source]
Return type:None
add_static(prefix: str, path: Union[pathlib.Path, str], **kwargs) → None[source]
Return type:None
url
logger = <Logger faust.web.base (WARNING)>
class faust.web.base.Request(*args, **kwargs)[source]

HTTP Request.

match_info
query
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, *, port: int = None, bind: str = None, **kwargs) → None[source]

Web server and framework implemention using aiohttp.

driver_version = 'aiohttp=3.4.0a0'
handler_shutdown_timeout = 60.0
text(value: str, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
html(value: str, *, status: int = 200) → faust.web.base.Response[source]
Return type:Response
json(value: Any, *, status: int = 200) → Any[source]
Return type:Any
bytes(value: bytes, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
route(pattern: str, handler: Callable) → None[source]
Return type:None
add_static(prefix: str, path: Union[pathlib.Path, str], **kwargs) → None[source]
Return type:None
logger = <Logger faust.web.drivers.aiohttp (WARNING)>
coroutine on_start(self) → None[source]

Called every time before the service is started/restarted.

Return type:None
coroutine start_server(self, loop: asyncio.events.AbstractEventLoop) → None[source]
Return type:None
coroutine stop_server(self, loop: asyncio.events.AbstractEventLoop) → None[source]
Return type:None
faust.web.site

Website served by the Faust web server.

class faust.web.site.Website(app: faust.types.app.AppT, *, port: int = None, bind: str = None, driver: Union[typing.Type[faust.web.base.Web], str] = 'aiohttp://', extra_pages: Sequence[Tuple[str, Type[faust.web.views.Site]]] = None, **kwargs) → None[source]

Service starting the Faust web server and endpoints.

pages = [('/graph', <class 'faust.web.apps.graph.Site'>), ('', <class 'faust.web.apps.stats.Site'>), ('/router', <class 'faust.web.apps.router.Site'>), ('/table', <class 'faust.web.apps.tables.Site'>)]
logger = <Logger faust.web.site (WARNING)>
init_driver(driver: Union[typing.Type[faust.web.base.Web], str], **kwargs) → None[source]
Return type:None
init_pages(extra_pages: Sequence[Tuple[str, Type[faust.web.views.Site]]]) → None[source]
Return type:None
faust.web.views

Class-based views.

class faust.web.views.View(app: faust.types.app.AppT, web: faust.web.base.Web) → None[source]

View (HTTP endpoint).

classmethod from_handler(fun: Callable[[faust.types.web.View, faust.types.web.Request], Awaitable[faust.types.web.Response]]) → Type[_ForwardRef('View')][source]
Return type:Type[View]
text(value: str, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
html(value: str, *, status: int = 200) → faust.web.base.Response[source]
Return type:Response
json(value: Any, *, status: int = 200) → faust.web.base.Response[source]
Return type:Response
bytes(value: bytes, *, content_type: str = None, status: int = 200) → faust.web.base.Response[source]
Return type:Response
route(pattern: str, handler: Callable) → Any[source]
Return type:Any
notfound(reason: str = 'Not Found', **kwargs) → faust.web.base.Response[source]
Return type:Response
error(status: int, reason: str, **kwargs) → faust.web.base.Response[source]
Return type:Response
coroutine delete(self, request: faust.web.base.Request) → Any[source]
Return type:Any
coroutine dispatch(self, request: Any) → Any[source]
Return type:Any
coroutine get(self, request: faust.web.base.Request) → Any[source]
Return type:Any
coroutine patch(self, request: faust.web.base.Request) → Any[source]
Return type:Any
coroutine post(self, request: faust.web.base.Request) → Any[source]
Return type:Any
coroutine put(self, request: faust.web.base.Request) → Any[source]
Return type:Any
class faust.web.views.Site(app: faust.types.app.AppT) → None[source]

Collection of HTTP endpoints (views).

enable(web: faust.web.base.Web, *, prefix: str = '') → List[faust.web.views.View][source]
Return type:List[View]
classmethod from_handler(path: str, *, base: Type[faust.web.views.View] = None) → Callable[Union[typing.Type[faust.types.web.View], typing.Callable[[faust.types.web.View, faust.types.web.Request], typing.Awaitable[faust.types.web.Response]]], Type[_ForwardRef('Site')]][source]
Return type:Callable[[Union[Type[View], Callable[[View, Request], Awaitable[Response]]]], Type[Site]]

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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

List agents.

title = 'Agents'
headers = ['name', 'topic', 'help']
sortkey = operator.attrgetter('name')
options = [<function option.<locals>.decorator>]
agents(*, local: bool = False) → Sequence[faust.types.agents.AgentT][source]
Return type:Sequence[AgentT[]]
agent_to_row(agent: faust.types.agents.AgentT) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, local: bool) → None[source]
Return type:None
faust.cli.base

Command-line programs using click.

faust.cli.base.argument(*param_decls, **attrs)[source]

Attaches an argument to the command. All positional arguments are passed as parameter declarations to Argument; all keyword arguments are forwarded unchanged (except cls). This is equivalent to creating an Argument instance manually and attaching it to the Command.params list.

Parameters:cls – the argument class to instantiate. This defaults to Argument.
faust.cli.base.option(*param_decls, **attrs)[source]

Attaches an option to the command. All positional arguments are passed as parameter declarations to Option; all keyword arguments are forwarded unchanged (except cls). This is equivalent to creating an Option instance manually and attaching it to the Command.params list.

Parameters:cls – the option class to instantiate. This defaults to Option.
class faust.cli.base.TCPPort[source]

CLI option: TCP Port (integer in range 1 - 65535).

name = 'range[1-65535]'
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 to PYTHONPATH, 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 passing imp=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
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>]
options = None
classmethod as_click_command() → Callable[source]
Return type:Callable
classmethod parse(argv: Sequence[str]) → Mapping[source]

Parse command-line arguments in argv’ and return mapping.

Return type:Mapping[~KT, +VT_co]
prog_name = ''
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:str
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:str
dark(text: str) → str[source]

Return cursor text.

Return type:str
bold(text: str) → str[source]

Return text in bold.

Return type:str
bold_tail(text: str, *, sep: str = '.') → str[source]

Put bold emphasis on the last part of a foo.bar.baz string.

Return type:str
say(*args, **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
carp(s: Any, **kwargs) → None[source]

Print something to stdout (or use file=stderr kwargs).

Note

Does not do anything if the --debug option is enabled.

Return type:None
dumps(obj: Any) → str[source]
Return type:str
coroutine run(self, *args, **kwargs) → Any[source]
class faust.cli.base.AppCommand(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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[_ForwardRef('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: Union[str, NoneType], key: str) → Any[source]

Convert command-line argument string to model (key).

Parameters:
  • typ (Optional[str]) – The name of the model to create.
  • key (str) – The string json of the data to populate it with.

Notes

Uses key_serializer to set the codec for the key (e.g. "json"), as set by the --key-serializer option.

Return type:Any
to_value(typ: Union[str, NoneType], value: str) → Any[source]

Convert command-line argument string to model (value).

Parameters:
  • typ (Optional[str]) – The name of the model to create.
  • key – The string json of the data to populate it with.

Notes

Uses value_serializer to set the codec for the value (e.g. "json"), as set by the --value-serializer option.

Return type:Any
to_model(typ: Union[str, NoneType], value: str, serializer: Union[faust.types.codecs.CodecT, str, NoneType]) → Any[source]

Convert command-line argument to model.

Generic version of to_key()/to_value().

Parameters:
  • typ (Optional[str]) – The name of the model to create.
  • key – The string json of the data to populate it with.
  • serializer (Union[CodecT, str, None]) – The argument setting it apart from to_key/to_value enables you to specify a custom serializer not mandated by key_serializer, and value_serializer.

Notes

Uses value_serializer to set the codec for the value (e.g. "json"), as set by the --value-serializer option.

Return type:Any
import_relative_to_app(attr: str) → Any[source]

Import string like “module.Model”, or “Model” to model class.

Return type:Any
to_topic(entity: str) → Any[source]

Convert topic name given on command-line to app.topic().

Return type:Any
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 is examples.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:str
faust.cli.completion
class faust.cli.completion.completion(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Output shell completion to be eval’d by the shell.

require_app = False
shell() → str[source]
Return type:str
coroutine run(self) → None[source]
Return type:None
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

List agents.

title = 'Agents'
headers = ['name', 'topic', 'help']
sortkey = operator.attrgetter('name')
options = [<function option.<locals>.decorator>]
agents(*, local: bool = False) → Sequence[faust.types.agents.AgentT][source]
Return type:Sequence[AgentT[]]
agent_to_row(agent: faust.types.agents.AgentT) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, local: bool) → None[source]
Return type:None
class faust.cli.faust.completion(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Output shell completion to be eval’d by the shell.

require_app = False
shell() → str[source]
Return type:str
coroutine run(self) → None[source]
Return type:None
class faust.cli.faust.model(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Show model detail.

headers = ['field', 'type', 'default*']
options = [<function argument.<locals>.decorator>]
model_fields(model: Type[faust.types.models.ModelT]) → Sequence[Sequence[str]][source]
Return type:Sequence[Sequence[str]]
field(field: faust.types.models.FieldDescriptorT) → Sequence[str][source]
Return type:Sequence[str]
model_to_row(model: Type[faust.types.models.ModelT]) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, name: str) → None[source]
Return type:None
class faust.cli.faust.models(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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>]
models(builtins: bool) → Sequence[Type[faust.types.models.ModelT]][source]
Return type:Sequence[Type[ModelT]]
model_to_row(model: Type[faust.types.models.ModelT]) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, *, builtins: bool) → None[source]
Return type:None
class faust.cli.faust.reset(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Delete local table state.

Warning

This command will result in the destruction of the following files:

  1. 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.

coroutine reset_tables(self) → None[source]
Return type:None
coroutine run(self) → None[source]
Return type:None
class faust.cli.faust.send(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, repeat: int = 1, min_latency: float = 0.0, max_latency: float = 0.0, **kwargs) → Any[source]
Return type:Any
class faust.cli.faust.tables(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

List available tables.

title = 'Tables'
coroutine run(self) → None[source]
Return type:None
class faust.cli.faust.worker(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Start ƒaust worker instance.

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>]
start_worker(logfile: str, loglevel: str, blocking_timeout: float, web_port: int, web_bind: str, web_host: str, console_port: int) → Any[source]
Return type:Any
banner(worker: faust.cli.worker.Worker) → str[source]

Generate the text banner emitted before the worker starts.

Return type:str
faust_ident() → str[source]
Return type:str
platform() → str[source]
Return type:str
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Show model detail.

headers = ['field', 'type', 'default*']
options = [<function argument.<locals>.decorator>]
model_fields(model: Type[faust.types.models.ModelT]) → Sequence[Sequence[str]][source]
Return type:Sequence[Sequence[str]]
field(field: faust.types.models.FieldDescriptorT) → Sequence[str][source]
Return type:Sequence[str]
model_to_row(model: Type[faust.types.models.ModelT]) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, name: str) → None[source]
Return type:None
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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>]
models(builtins: bool) → Sequence[Type[faust.types.models.ModelT]][source]
Return type:Sequence[Type[ModelT]]
model_to_row(model: Type[faust.types.models.ModelT]) → Sequence[str][source]
Return type:Sequence[str]
coroutine run(self, *, builtins: bool) → None[source]
Return type:None
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Delete local table state.

Warning

This command will result in the destruction of the following files:

  1. 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.

coroutine reset_tables(self) → None[source]
Return type:None
coroutine run(self) → None[source]
Return type:None
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = 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, repeat: int = 1, min_latency: float = 0.0, max_latency: float = 0.0, **kwargs) → Any[source]
Return type:Any
faust.cli.tables

Program faust tables used to list tables.

class faust.cli.tables.tables(ctx: click.core.Context, *args, key_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

List available tables.

title = 'Tables'
coroutine run(self) → None[source]
Return type:None
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, NoneType] = None, value_serializer: Union[faust.types.codecs.CodecT, str, NoneType] = None, **kwargs) → None[source]

Start ƒaust worker instance.

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>]
start_worker(logfile: str, loglevel: str, blocking_timeout: float, web_port: int, web_bind: str, web_host: str, console_port: int) → Any[source]
Return type:Any
banner(worker: faust.cli.worker.Worker) → str[source]

Generate the text banner emitted before the worker starts.

Return type:str
faust_ident() → str[source]
Return type:str
platform() → str[source]
Return type:str

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 previous releases, please 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.

  • 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: 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.

  • 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.26

release-date:2018-07-30 08:00 A.M PDT
release-by:Ask Solem
  • Public release.

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 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 using web.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.

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

  • 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 virtual env 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 where table._revivers
    is an empty list.
  • 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

  • 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

  • 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 “hardcoded 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

  • 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 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

  • App: key_type and value_type can now be set to:

    • int: key/value is number stored as string
    • float: 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 out AgentManager 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 whe 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:

  • 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 spinloop.

  • 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 example Mock(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:

  • 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 to AgentManager.

      The agent manager does not use the app, but fixing this in anticipation of people writing custom agent managers.

    • AgentManager: On partitions revoked

      the 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 raises StopAsyncIteration, 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.

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) where n 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 Acknowledgement

    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 “acknowledgement” 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 the within 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.

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 codebases, 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.

  1. Create a GitHub account.

You need to create a GitHub account to be able to create new issues and participate in the discussion.

  1. 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.

  1. 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.

  1. 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.

  1. 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.

  1. If the error is from a Python traceback, include it in the bug report.

  2. 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.

  3. 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:

    • Collect tracing data using strace`_(Linux), :command:`dtruss (macOS), and ktrace (BSD), ltrace, and lsof.
  4. 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.

  1. 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:

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 example v2.3.1.
  • Experimental releases contain an additional identifier vX.Y.Z-id – for example v3.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 corresponding faust.types.app.AppT; for faust.Channel there is a faust.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
    else:
        class App: ...  # noqa
    
  • 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.

  1. 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
    
  2. Modify setup.py

    After the requirements file is added you need to add it as an option to setup.py in the EXTENSIONS section:

    EXTENSIONS = {
        'debug',
        'fast',
        'rocksdb',
        'uvloop',
    }
    
  3. 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:

    http://readthedocs.org/projects/faust/?fromdocs=faust

  • 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.0
Date:Aug 17, 2018

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 apps 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.Tasks via self.add_future. These dependencies will be started/stopped/restarted with the service.

Since the Service class requires the asyncio loop at start, there’s also ServiceProxy. This special subclass is used by App and Agent as they are created at module time, for example the module t.py:

# t.py
import faust

app = faust.App('myid')

The ServiceProxy makes the initialization of the service part lazy, and delegates all service methods to a composite class (App -> AppService, Agent -> AgentService).

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.

Webserver

The Worker also starts the web service. This is a local web server running in each Faust instances that exposes things like statistics and access to table data.

App

The app configures the Faust instance, and is the entrypoint 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 tha), 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.

Table

Any user defined table.

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.

Stream

These are individual streams, started after everything is set up.

Fetcher

The Fetcher is the service that actually retrieves messages from the kafka topic. The fetcher forwards these messages to the TopicManager, which in turns forwards it to Topic’s and streams.

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:

  1. Check all repartition source topics and use internal topic manager to make sure they have been created with the right number of partitions.

  2. 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.

  3. 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.

  4. 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).

StickyAssignor

With our simple PartitionGrouper we can use a StickyPartitionAssignor to assign partitions to the clients. However we need to explicitly handle standby assignments here. We use the StickyPartitionAssignor design approved in KIP-54 as the basis for our sticky assignor.

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.0.

Release:1.0
Date:Aug 17, 2018

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.0.

0.9.65
release-date:2018-04-27 2:04 P.M PDT
release-by:Vineet Goel
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() and to_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:

  • Consumer: Fixed asyncio.base_futures.IllegalStateError error in commit handler.

  • CLI: Fixed bug when invoking worker using faust -A.

Authors

Creators

Name Email
Ask Solem <ask@robinhood.com>
Vineet Goel <vineet@robinhood.com>

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.

Comitters

Arpan Shah <arpan@robinhood.com>
Sanyam Satia <sanyam@robinhood.com>

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 <allison.wang@robinhood.com>
Jamshed Vesuna <jamshed@robinhood.com>
Jaren Glover <jaren@robinhood.com>
Jerry Li <jerry.li@robinhood.com>
Prithvi Narasimhan <narasimhan.prithvi@gmail.com>
Ruby Wang <ruby.wang@robinhood.com>
Shrey Kumar Shahi <shrey@robinhood.com>

Glossary

acked
acking
acknowledged
Acknowledgement 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
nillipotence
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.

Indices and tables