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

.. sourcecode:: python

    app = faust.App('myapp', broker='kafka://localhost')

    # Models describe how messages are serialized:
    # {"account_id": "3fae-...", amount": 3}
    class Order(faust.Record):
        account_id: str
        amount: int

    @app.agent(value_type=Order)
    async def order(orders):
        async for order in orders:
            # process infinite stream of orders.
            print(f'Order for {order.account_id}: {order.amount}')

The Agent decorator defines a "stream processor" that essentially
consumes from a Kafka topic and does something for every event it receives.

The agent is an ``async def`` function, so can also perform
other operations asynchronously, such as web requests.

This system can persist state, acting like a database.
Tables are named distributed key/value stores you can use
as regular Python dictionaries.

Tables are stored locally on each machine using a super fast
embedded database written in C++, called `RocksDB`_.

Tables can also store aggregate counts that are optionally "windowed"
so you can keep track
of "number of clicks from the last day," or
"number of clicks in the last hour." for example. Like `Kafka Streams`_,
we support tumbling, hopping and sliding windows of time, and old windows
can be expired to stop data from filling up.

For reliability we use a Kafka topic as "write-ahead-log".
Whenever a key is changed we publish to the changelog.
Standby nodes consume from this changelog to keep an exact replica
of the data and enables instant recovery should any of the nodes fail.

To the user a table is just a dictionary, but data is persisted between
restarts and replicated across nodes so on failover other nodes can take over
automatically.

You can count page views by URL:

.. sourcecode:: python

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

.. sourcecode:: python

    # 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 :pypi:`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** :ref:`introduction` **introduction page**
    to read more about Faust, system requirements, installation instructions,
    community resources, and more.

**or go directly to the** :ref:`quickstart` **tutorial**
    to see Faust in action by programming a streaming application.

**then explore the** :ref:`User Guide <guide>`
    for in-depth information organized by topic.

.. _`Robinhood`: http://robinhood.com
.. _`async/await`:
    https://medium.freecodecamp.org/a-guide-to-asynchronous-programming-in-python-with-asyncio-232e2afa44f6
.. _`Celery`: http://celeryproject.org
.. _`Kafka Streams`: https://kafka.apache.org/documentation/streams
.. _`Apache Spark`: http://spark.apache.org
.. _`Storm`: http://storm.apache.org
.. _`Samza`: http://samza.apache.org
.. _`Flink`: http://flink.apache.org
.. _`RocksDB`: http://rocksdb.org
.. _`Apache Kafka`: https://kafka.apache.org
