Skip to content

Releases: ArroyoSystems/arroyo

v0.13.0

19 Dec 16:35
Compare
Choose a tag to compare

These release notes can also view viewed on the Arroyo blog

This release introduces support for metadata from sources, a new RabbitMQ connector, improved CDC support, IAM auth for Kafka, a more efficient core dataflow, among many other improvements.

Arroyo is a community project, and we're very grateful to all of our contributors. We are particularly excited to welcome four new contributors to the project in this release:

Thanks to all of our contributors for this release:

Features

Source metadata

In Arroyo, users write SQL DDL statements (CREATE TABLE) to define the schema of incoming data from sources. But the underlying data in those sources is in some format or encoding, like JSON or Avro. Arroyo deserializes the data, and maps the fields in the data into the SQL schema so it can be used in queries.

So that works for data. But there are other bits of context that users might want to be able to use as well, not part of the actual data of the message but related metadata. For example, in Kafka a user might want to access the partition of a message, or the offset, or timestamp.

Previously this was not possible, but in 0.13 we have added support for metadata fields in source tables. It looks like this:

create table users (
    id TEXT,
    name TEXT,
    offset BIGINT GENERATED ALWAYS AS (metadata('offset_id')) STORED,
    partition INT GENERATED ALWAYS AS (metadata('partition')) STORED
) with (
    connector = 'kafka',
    ...
);

To access metadata fields, you can now define a generated column on the source table that is defined with the special function metadata. This takes a single string-literal argument that names the metadata field that should be injected into that column.

Initially, source metadata is supported in the Kafka (offset_id, partition, topic, and timestamp), and MQTT (topic) connectors.

Thanks to @vaibhawvipul for this incredible contribution!

RabbitMQ Streams connector

RabbitMQ is a message broker that supports a wide variety of event-processing patterns and protocols. Arroyo has long supported its MQTT mode via our MQTT connector. Now in 0.13 we are adding support for its native
Streams protocol, which adds capabilities around replay and persistence to support at-least-once semantics for processing.

A RabbitMQ source table looks like this

create table stream (
    user_id INT,
    item TEXT,
    price FLOAT,
    timestamp TIMESTAMP
) with (
    connector = 'rabbitmq',
    host = 'localhost',
    stream = 'orders',
    type = 'source',
    format = 'json'
);

See the connector docs for all the details on how to configure and use the source.

Thanks to @ecarrara for contributing this new source to the project!

Atomic update outputs

Arroyo supports two forms of streaming SQL, which we call dataflow semantics and update semantics. Update queries are modeled as materialized views which incrementally update as new events come in.

A simple example is a count query:

SELECT count(*) FROM events;

In a batch system, running this query would return the total number of records in the table. But in a streaming system like Arroyo, records come in indefinitelyβ€”to produce a result for count we'd need to wait forever. A query engine that never returns a result isn't very useful, so instead we periodically report the result based on the events we've seen so far.

But how will we report this? In this case there's a single value that we're reporting, so we could just emit an updating count, like

count(*)  10
count(*)  20
count(*)  30
...

Our goal is to allow downstream systems to consume these updates and keep track of the current state of the table. For more complex queries that have multiple rows, we need to tell the consumer which row we're updating. There are three kinds of updates we need to be able to handle: creates, updates, and deletes.

For example, for a query like this one

SELECT user_id, count(*) as count
FROM events
GROUP BY user_id
HAVING count < 10;

we would emit a create when first encountering an event for a user_id, updates on subsequent events, and finally a delete once they accumulated more than 10 events.

At least, in theory. In practice, the internal details of our implementation could not actually handle updates. Instead, we modeled updates as a delete followed by a create. This is not only inefficient, but it's also non-atomic: if the system crashes between the delete and the create, the downstream system be left in an inconsistent state.

This is now addressed in 0.13; internally Arroyo can now correctly ingest updates from CDC sources, model them internally, and emit them to sinks. So in 0.12, that first query would return (in the Debezium JSON format we use)

{"before":null,"after":{"count(*)":10},"op":"c"}
{"before":{"count(*)":10},"after":null,"op":"d"}
{"before":null,"after":{"count(*)":20},"op":"c"}
{"before":{"count(*)":20},"after":null,"op":"d"}
{"before":null,"after":{"count(*)":30},"op":"c"}
{"before":{"count(*)":30},"after":null,"op":"d"}
{"before":null,"after":{"count(*)":40},"op":"c"}

while in 0.13 this becomes

{"before":null,"after":{"count(*)":10},"op":"c"}
{"before":{"count(*)":10},"after":{"count(*)":20},"op":"u"}
{"before":{"count(*)":20},"after":{"count(*)":30},"op":"u"}
{"before":{"count(*)":30},"after":{"count(*)":40},"op":"u"}

There is one breaking change associated with this: for Debezium CDC sources, it's now required to mark primary keys so that we can properly apply this logic. It looks like this:

CREATE TABLE debezium_source (
    id INT PRIMARY KEY,
    customer_id INT,
    price FLOAT,
    order_date TIMESTAMP,
    status TEXT
) WITH (
    connector = 'kafka',
    format = 'debezium_json',
    type = 'source',
    ...
);

IAM Auth for Kafka

There are many, many ways to authenticate with Kafka. You can use a SASL username and password. You can use an SSL private key. Mutual TLS. The list goes on.

Or if you are using one of AWS's Managed Streaming for Kafka (MSK) productsβ€”in particularly its serverless offeringβ€”you must authenticate via a bespoke IAM protocol.

Thanks to contributor @emef, this protocol is now supported in Arroyo, simplifying the process of using it with MSK.

To use it, simply specify AWS_MSK_IAM as the authentication protocol and the AWS region:

Kafka MSK creation

Or in SQL:

CREATE TABLE msk (
  ...
) WITH (
  connector = 'kafka',
  'auth.type' = 'aws_msk_iam',
  'auth.region' = 'us-east-1',
  ...
)
  • support AWS_MSK_IAM authentication by @emef in #789

Operator chaining

If you've made it this far, I'm going to assume you're pretty interested in stream processing engines. Or you just scrolled around for a while before landing here. Either way, we're going to get deep into the details

Arroyo 0.13 introduces a new feature into the core dataflow: operator chaining. What is operator chaining? First we need to understand a bit about the physical structure of a stream processing pipeline. A user-supplied SQL query defining a pipeline goes through several stages of transformation, from SQL text, to a logical plan, and finally to a physical dataflow graph.

The dataflow graph is what we actually execute. Each node is an operator, which consumes data, does some (possibly stateful) transformation, and produces an output; oper...

Read more

v0.12.1

04 Nov 17:50
Compare
Choose a tag to compare

v0.12.1 is a patch release containing several fixes on top of 0.12.0. It is a drop-in replacement for clusters running 0.12.0 and is recommended for all users.

  • Properly return a scalar value from a UDF when only input is scalar by @mwylde in #753
  • Urlencode subject in confluent schema registry URLs by @emef in #759
  • Skip the whole confluent schema header in proto deser by @emef in #763
  • Update object_store to pull in GCS token refresh fix by @mwylde in #770
  • Plan and optimize generating expressions by @mwylde in #778
  • Properly plan subqueries with async UDFs by @mwylde in #780

Full Changelog: v0.12.0...v0.12.1

v0.12.0

24 Sep 15:25
Compare
Choose a tag to compare

These release notes are also available on the Arroyo blog

The Arroyo team is thrilled to announce that Arroyo 0.12.0 is now available! This release introduces Python UDFs, which allow Python developers to extend the engine with custom functions, callable from SQL. We've also added support for Protobuf as an ingestion format, new JSON syntax, custom state TTLs for updating SQL queries, among many other features, improvements, and fixes.

Excited to try things out? Getting started is easier than ever with new native packages for Linux and MacOS, complementing our existing Docker images and Helm chart.

Arroyo is a community project, and we're very grateful to all of our contributors. We are particularly excited to welcome four new contributors to the project in this release:

Thanks to all of our contributors for this release:

And now, all of the details on what's new in Arroyo 0.12!

Features

Python UDFs

Arroyo has long supported user-defined functions (UDFs), allowing users to extend the engine by writing new scalar, aggregate, and async functions. We've been amazed by the diversity of UDFs that our users have come up with, including

  • Parsers for custom formats
  • Ad-hoc joins with other databases
  • Calling AI inference APIs
  • Sinks to other data systems
  • Integrating specialized financial libraries

Among many other use cases. But until now, Arroyo only supported UDFs written in Rust. We love Rust, but we know it's not the most popular (or second, or third, or...) language for data users.

So in 0.12, we're thrilled to support UDFs written in Python.

It looks like this

from arroyo_udf import udf

@udf
def levenshtein(s1: str, s2: str) -> int:
    if len(s1) < len(s2):
        return levenshtein(s2, s1)

    if len(s2) == 0:
        return len(s1)

    previous_row = range(len(s2) + 1)

    for i, c1 in enumerate(s1):
        current_row = [i + 1]
        for j, c2 in enumerate(s2):
            insertions = previous_row[j + 1] + 1
            deletions = current_row[j] + 1
            substitutions = previous_row[j] + (c1 != c2)
            current_row.append(min(insertions, deletions, substitutions))
        previous_row = current_row

    return previous_row[-1]

which can then be used in SQL

SELECT levenshtein(username, email) as distance
from events;

Python UDFs take a series of arguments, each of which can be called with a SQL column or literal. The argument types and return type are determined by the function signature and type hints, including support for Optional to indicate how nullability should interact with the UDF.

We've also updated the Web UI to add a Python UDF editor.

What we're releasing in 0.12 is just the start. In our next release, we will add support for Python UDAFs, as well as direct PyArrow support for high-performance Python integrations without deserialization or copying overhead.

For more details on Python UDFs, see the documentation.

We're excited to see what you build with Python UDFs!

Protobuf

Protocol buffersβ€”better known as protobufβ€”is a fast, space-efficient binary data format that's commonly used in event pipelines. And as of Arroyo 0.12, it's now natively supported as an ingestion format, along with support for reading protobuf schemas from Confluent Schema Registry.

This expands on our existing set of formats, including JSON, Avro, and raw string and bytes.

All protobuf features are supported, including optionals, lists, structs, enums, and imports.

See the full format documentation here.

  • Protobuf deserialization support by @mwylde in #715
  • Add confluent schema registry support for protobuf by @mwylde in #724
  • Add protoc to docker image for protobuf support by @mwylde in #725

JSON syntax

Sometimes you don't have a nice, proper schema for the JSON flowing through your data pipelines (it's ok, we've all been there). Arroyo still has you covered, with unstructured JSON fields (type JSON). And now the experience is even better, thanks to a suite of new JSON functions and integration of Postgres-style JSON syntax.

It looks like this:

CREATE TABLE events (
  value JSON
) WITH (
  connector = 'kafka',
  bootstrap_servers = 'kafka:9092',
  topic = 'events',
  format = 'json',
  type = 'source',
  'json.unstructured' = 'true'
);

SELECT
  -- using the json_get function
  json_get(value, 'user', 'name')::TEXT as name,
  -- or using the -> operator
  value->'user'->'email' as email,
  -- field presence check can be done with the ? operator
  value ? 'id' as has_id
FROM events;

There are several ways to access JSON fields:

  • json_get(json: str, *keys: str | int) takes a JSON-encoded string and a series of keys to traverse,
    returning a partially-parsed JSON value that can be further processed without needing to be re-parsed
  • The Postgres-style -> operator is a synonym for json_get, and can be efficiently chained
  • json_get_{str|int|bool|float|json}(json: str, *keys: str | int) is a set of convenience functions
    that return the JSON value as a specific type
  • The --> operator is a synonym for json_get_str
  • SQL type casts can also be used with json_get to get an output of the desired type, like
    json_get(value, 'a')::INT

We've also added a couple other handy functions for working with JSON:

  • json_contains(json: str, *keys: str | int) (aliased to the ? operator)
  • json_length(json: str, *keys: str | int) -> int

Under the hood, these new functions use the ultra-fast JSON parser
jiter and deserialize data into an efficient
parsed format, avoiding the need the repeatedly re-parse data to access multiple fields.

See the json function docs for more detail.

  • Add new json functions from datafusion-functions-json by @mwylde in #640

Custom State TTLs

Arroyo has two intersecting streaming SQL semantics, which we call dataflow SQL and updating SQL. Dataflow SQL is based around time-oriented windows, which encode a notion of completeness via watermark-based processing. In other words, for a particular window, the watermark tells us that we can process it and then drop the data for that window from our state.

But updating semantics have no in-built notion of completeness. These are queries like

SELECT user, count(*)
FROM events
GROUP BY user;

The semantics of the query are that, for every user, we should be able to output the complete count of their events going back to the beginning of time. But it's generally intractable in a streaming system to actually keep all of the data for all time without blowing up our state.

To make these queries tractable, Arroyo supports a TTL (time-to-live) for updating state, which controls how long we will keep data around after seeing a particular key (like the user/count pair in that example). In previous Arroyo releases this was configurable only at the cluster level, but now it can be modified at a per-query level with SET updating_ttl.

So if we want to keep the state around for longer, we can write:

SET updating_ttl = '7 days';

SELECT user, count(*)
FROM events
GROUP BY user;

IRSA support

AWS has a powerful (and achingly complex) system for managing permissions across services called IAM. Kubernetes has a completely different access-control system based on roles and service accounts.

So...how do you manage permissions when running a Kubernetes cluster on AWS? For example if you wanted to run a certain stream processing engine that, perhaps, needs to access an S3 bucket?

The answer is IAM Roles for Service Accounts (IRSA), a predictable mouthful of an acronym from the marketing folks who brought you the streaming service called Kinesis Data Analytics for Apache Flink.

But naming aside, IRSA lets you attach an IAM role to a Kubernetes service account. And in 0.12, it's now fully supported in Arroyo. This provides a secure, granular way to control Arroyo's access to your AWS resources.

Sett...

Read more

v0.11.3

20 Aug 17:36
Compare
Choose a tag to compare

v0.11.3 is a patch release containing several fixes on top of 0.11.2. It is a drop-in replacement for clusters running any 0.11 patch release.

What's changed

  • Clean up backup files (#706)
  • Improve robustness of SSE source to server EOFs (#711)
  • Fix race condition that could stall scheduling if operator panicked during setup (#712)
  • Update DF fork to 4f17b2b364d for metrics memory leak fix (#717)

v0.11.2

30 Jul 03:27
Compare
Choose a tag to compare

v0.11.2 is a patch release containing several fixes on top of 0.11.1. It is a drop-in replacement for clusters running 0.11.0 or 0.11.1.

What's changed

  • Fix for checkpoint cleanup failure (#689)
  • Use correct (relative) delta paths when writing to object stores (#693)
  • Add support for IRSA authentication for S3 (#694)

Full Changelog: v0.11.1...v0.11.2

v0.11.1

12 Jul 23:34
Compare
Choose a tag to compare

0.11.1 is a patch release containing several bugfixes on top of 0.11.0. It is a drop-in replacement for clusters running 0.11.0.

What's changed

  • Fix regression in updating equi-join optimization (#686)
  • Add option to set raw_datums in avro schema editor (#685)
  • Fix pending requests in preview that could stall out the webui (#684)
  • Use an inner join to prevent controller from loading partially-constructed jobs (#681)

Full Changelog: v0.11.0...v0.11.1

v0.11.0

02 Jul 18:52
Compare
Choose a tag to compare

These release notes are also available on the Arroyo blog.

The Arroyo team is very excited to announce the release of Arroyo 0.11, which includes a number new features and improvements on top of 0.10, including pipeline clusters, a new, lightweight way to run streaming pipelines on serverless infrastructure, sqlite support for the control plane, a new configuration system, refinement of the create pipeline UX, and more.

In this release, we are thrilled to welcome two new contributors to the project:

Thanks to all of our contributors for this release:

Excited to try things out? Download a native package or run in Docker with

$ docker run -p 5115:5115 ghcr.io/arroyosystems/arroyo:0.11

Pipeline clusters

Today Arroyo operates as a distributed cluster, with a controller node that manages a collection of pipelines, which themselves are scheduled on Arroyo nodes or Kubernetes.

Arroyo 0.11 introduces a new, lighter-weight way to run individual jobs, which we're calling pipeline clusters.

It works like this:

CREATE TABLE coinbase (
    type TEXT,
    price TEXT
) WITH (
    connector = 'websocket',
    endpoint = 'wss://ws-feed.exchange.coinbase.com',
    subscription_message = '{
      "type": "subscribe",
      "product_ids": [
        "BTC-USD"
      ],
      "channels": ["ticker"]
    }',
    format = 'json'
);

select avg(cast(price as FLOAT)) from coinbase
group by hop(interval '1 second', interval '10 seconds');

Here we've created a SQL file containing an Arroyo query and run it directly with the arroyo binary with the new arroyo run command. This starts up a completely self-contained minicluster that starts and manages the provided query. When the process is signalled to exit (via ctrl-c or SIGINT) the pipeline is stopped with a final checkpoint so it can be resumed later. If a sink isn't specified (as an INSERT INTO statement), the default STDOUT sink is used, which means you can consume pipeline outputs with UNIX pipes:

$ arroyo run query.sql > outputs.json

Pipeline clusters are great for running pipelines locally or in CI. But they also make it possible to run streaming pipelines in light-weight container runtimes like Amazon Fargate, Google Cloud Run, and Fly.io.

When running on a container runtime, state can be snapshotted and restored from an object store like S3 or GCS with the --state-dir flag. This means stateful, consistent, and reliable processing even on transient resources.

See the pipeline cluster docs for more details, and watch out for more tutorials and guides to running Arroyo on various serverless runtimes.

Sqlite control plane

In 0.10, we slimmed down the architecture of Arroyo, from a half-a-dozen services to threeβ€” one Arroyo service plus Postgres for configuration and metadata, and Prometheus to power the metrics features of the Web UI. In 0.11 we've finished this work:

  • Sqlite is now supported as the storage engine for the control plane, powering the pipeline configuration
    and API
  • We've replaced prometheus with a small, purpose-built metrics system to power the built-in metrics features
    (with prometheus-compatible metrics still available for integration with external observability systems)

The entire Arroyo system can now run off a single, statically-linked, 150MB binary.

This greatly reduces the complexity and resource requirements of a minimal Arroyo deployment, which can now run comfortably in lightweight container hosting.

  • Add Sqlite support for the control plane by @mwylde in #626
  • Support metrics for web UI natively instead of using prometheus by @mwylde in #630
  • Migrate arroyo-bin cmd to arroyo and consolidate docker images by @mwylde in #659

Configuration

Arroyo 0.11 ships with a completely new configuration system, replacing the ad-hoc environment variables used previously. This makes the system easier and more consistent to customize. We've also added comprehensive documentation on the various configuration options.

The new system is hierarchical; at the base is a
default config shipped with the engine. This can be overridden by Toml or Yaml files placed in the Users or system config directory (on Linux for example ~/.config/arroyo/config.toml), an arroyo.toml file in the current directory, then a config file passed on the command line with the new --config option. Finally, everything can be overridden by environment variables.

For example, on my Mac I have the following local config in my ~/Library/Application Support/arroyo/config.toml file

checkpoint-url = "/Users/mwylde/arroyo/checkpoints"
artifact-url = "/Users/mwylde/arroyo/artifacts"

[database]
type = "sqlite"

On a per-run basis I can override this with environment variables, for example, to use Postgres instead

$ ARROYO__DATABASE__TYPE=postgres arroyo cluster

See the configuration docs for all of the options and full details on how the configuration system works.

Existing configuration environment variables (like SCHEDULER, DATABASE_HOST, etc.) will continue to be supported with a warning in 0.11, but will be removed in 0.12.

Create pipeline and preview improvements

We've reworked the UX for creating a pipeline and previewing it in the Web UI, focusing on performance and increased information density.

We've made the sidebars collapsible and integrated other UI elements to increase the amount of vertical and horizontal space for your queries and results.

The result table is now a high-performance virtual renderer, allowing us to increase the number of results shown to 10,000 (from 20 today), and the columns are now resizable, re-orderable, and sortable.

For power users there are few new options, including a checkbox to enable sinks in preview (by default they are disabled to avoid polluting external systems with test data) and set the parallelism on pipeline start.

  • Redesign create pipeline UX and pipeline outputs for greater density and performance by @mwylde in #663

Raw bytes

Arroyo supports a number of formats (including JSON, Avro, and Parquet) but there will always be a long-tail of formats (potentially company-internal) that aren't built in.

Now it's possible to build your own decoders using UDFs and the new raw_bytes format.

When I started in streaming, the classic demo was analyzing Twitter data to find popular hashtags. Unfortunately, today Twitter API access costs thousands of dollars a month and is locked down. Fortunately there are now more open Twitter alternatives, like Bluesky which publishes all of its messages using the open AT protocol. And a website called Firesky has already done the hard work of aggregating these events into a websocket feed.

That would be great for Arroyo, except that its data is encoded in a binary format called cbor, a kind of binary JSON.

With raw_bytes and the serde-json crate this is no longer an issue!

We'll start with a simple UDF that converts cbor to JSON:

/*
[dependencies]
serde_cbor = "0.11"
serde_json = "1"
serde = {version = "1", features = ["derive"]}
serde-transcode = "1"
*/

use arroyo_udf_plugin::udf;


#[udf]
fn cbor_to_json(data: &[u8]) -> Option<String> {
    let mut deserializer = serde_cbor::Deserializer::from_slice(data);
    let mut buf = std::io::BufWriter::new(Vec::new());
    let mut serializer = serde_json::Serializer::new(&mut buf);
    serde_transcode::transcode(&mut deserializer, &mut serializer).ok()?;
    let bytes = buf.into_inner().unwrap();

    Some(String::from_utf8(bytes).ok()?)
}

(we could have also processed directly on the cbor data, but the tools for dealing with JSON are a bit more convenient).

Then, we can use this in a query to count the hashtags:

create table firesky (
    value BYTEA
) with (
    connector = 'websocket',
    endpoint = 'wss://firesky.tv/ws/app',
    format = 'raw_bytes'
);

create view tags as (
    s...
Read more

v0.10.3

11 Jun 16:03
Compare
Choose a tag to compare

0.10.3 is a patch release containing a couple of fixes on top of 0.10.2. It is a drop-in replacement for clusters running any patch release of 0.10.

What's changed

  • Add table for kafka exactly-once state by @mwylde in #655
  • Add option to deploy kubernetes programs as configmaps rather than env vars by @mwylde in #654

Full Changelog: v0.10.2...v0.10.3

Note that to take advantage of the fix for the issue in #654, the helm chart must be upgraded to 0.10.3 as well (the existing helm chart will work with the new image, however, the fix is disabled as it requires additional k8s permissions).

v0.10.2

08 May 18:37
Compare
Choose a tag to compare

0.10.2 is a patch release containing several fixes on top of 0.10.1. It is a drop-in replacement for clusters running 0.10.0 or 0.10.1.

What's Changed

Full Changelog: v0.10.1...v0.10.2

v0.10.1

01 May 17:03
Compare
Choose a tag to compare

0.10.1 is a patch release containing several fixes on top of 0.10.0. It is a drop-in replacement for clusters running 0.10.0.

Full Changelog: v0.10.0...v0.10.1