Skip to content

Releases: ArroyoSystems/arroyo

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

v0.10.0

25 Apr 17:18
Compare
Choose a tag to compare

These release notes are also available on the Arroyo blog.

Arroyo 0.10.0 is a major release of the Arroyo stream processing engine, featuring an entirely new SQL engine that's >3x faster and ships as a single binary. Plus NATS and MQTT connectors, more SQL features, and more.

NATS

NATS is a messaging and queueing system designed for simplicity and performance. Arroyo 0.10 adds sources and sinks for Core NATS and NATS Jetstream, which layers on persistence and delivery guarantees.

For example, to consume from a NATS Jetstream subject, we can use this DDL:

CREATE TABLE logs (
    id BIGINT NOT NULL,
    time TIMESTAMP NOT NULL,
    host TEXT NOT NULL,
    level TEXT NOT NULL,
    message TEXT NOT NULL
) with (
    type = 'source',
    connector = 'nats',
    servers = 'localhost:4222',
    'nats.subject' = 'logs',
    'auth.type' = 'credentials',
    'auth.username' = '{{ NATS_USER }}',
    'auth.password' = '{{ NATS_PASSWORD }}',
    format = 'json'
);

See the connector docs for more details.

Thanks to Quentin Gaborit for this incredible contribution!

  • Add NATS source and sink connectors by @gbto in #578

MQTT

MQTT is a lightweight messaging protocol widely used with low-power devices and in the Internet of Things (IoT) space. Arroyo 0.10 now ships with an MQTT source and sink to consume and produce from MQTT brokers.

For example, you can create an MQTT source with the following SQL

CREATE TABLE devices (
    device_id BIGINT NOT NULL,
    time TIMESTAMP NOT NULL,
    lat FLOAT NOT NULL,
    lng FLOAT NOT NULL,
    metadata JSON
) with (
    connector = 'mqtt',
    url = 'tcp://localhost:1883',
    type = 'source',
    topic = 'events',
    format = 'json'
);

See the connector docs for more details.

Thanks to community member Giovanny Gutiérrez (@bakjos) for this amazing contribution!

$ arroyo

With the new architecture in 0.10, the entire Arroyo system now builds as a single binary

$ arroyo
Usage: arroyo <COMMAND>

Commands:
  api         Starts an Arroyo API server
  controller  Starts an Arroyo Controller
  cluster     Starts a complete Arroyo cluster
  worker      Starts an Arroyo worker
  compiler    Starts an Arroyo compiler
  node        Starts an Arroyo node server
  migrate     Runs database migrations on the configure Postgres database
  help        Print this message or the help of the given subcommand(s)

Options:
  -h, --help     Print help
  -V, --version  Print version

Running a complete local cluster is as easy as

$ arroyo cluster
INFO arroyo_controller: Using process scheduler
INFO arroyo_server_common: Starting cluster admin server on 0.0.0.0:8001
INFO arroyo_controller: Starting arroyo-controller on 0.0.0.0:9190
INFO arroyo_api: Starting API server on 0.0.0.0:8000
INFO arroyo_compiler_service: Starting compiler service at 0.0.0.0:9000

Today we are providing pre-compiled binaries on the release page for Linux and MacOS, or you can build your own by following the dev instructions.

Note that Arroyo currently requires a running Postgres database for configuration, but in the next release we will be adding the option of using sqlite. Prometheus is also used to power the pipeline metrics in the Web UI if available.

  • Add a new binary crate 'arroyo-bin' to package the entire system by @mwylde in #514

Performance

Arroyo 0.10 is significantly faster than 0.9.

In fact, it's so fast that for many benchmarks we've run on EC2 instances, it can saturate the network before using all the available CPU. We'll be following up with rigorous benchmarks against other systems, but in early tests it significantly beats other streaming systems in throughput.

How has it gotten so much faster? This is deserving of an entire blog post, but the short version is that Arroyo now operates internally on columnar data using carefully tuned vector kernels, thanks to the hard work of the Apache Arrow and DataFusion communities.

Columnar data is now the standard for OLAP (analytics-oriented) query engines like ClickHouse, Pinot, and Presto. There are a few reasons for this:

  • By storing all values in a column together, you can achieve better compression ratios and make better use of CPU cache
  • Only the columns actually referenced in a query need to be read, reducing disk and network IO
  • Columnar processing aligns well with the vector capabilities in modern CPUs, providing 3x or more speedups

However, row-oriented data remains the standard for streaming engines. There are some inherent tradeoffs between latency (how quickly an event can traverse through the pipeline) and throughput (how many events can be processed with a given amount of CPU). By batching data we can get higher throughput at the expense of latency. And columnar representations require that we batch a number of events together before we see performance improvements (in fact, with a small number of rows columnar processing will be much slower due to fixed overhead).

But there's a strong argument for moving to columnar representations for streaming: at any given batch size, the higher the throughput the less time we must wait to fill that batch. For example, if we want at least 100 records in our batch to overcome fixed costs, the amount of time we need to wait to receive 100 records will depend on our throughput:

  • At 10 events/second, it takes 1 second
  • At 1,000 — 0.01 seconds (100ms)
  • At 1,000,000 — 0.0001 (0.1ms)

Or looking at it from a fixed latency perspective (say, waiting at most 10ms):

  • At 10 events/second, our batch size is 1
  • At 1,000 — 100
  • At 1,000,000 — 100,000

In Arroyo 0.10, we perform batching at the source and via aggregating operators like windows and joins. The source batching behavior can be configured via two environment variables:

  • BATCH_SIZE which controls the maximum batch size
  • BATCH_LINGER_MS which controls the maximum amount of time to wait (in milliseconds) before emitting a batch

By configuring these values, users can choose their own tradeoff between latency and throughput.

SQL Improvements

Functions

We've added support for over 200 new scalar, aggregate, and window SQL functions. These unlock many powerful new capabilities that previously would have been inexpressible or would have required writing a UDF.

There are too many new functions to highlight them all, but some exciting additions include:

  • Nearly 200 scalar functions, covering math, conditions, strings, regex, JSON, times, and more
  • Statistical and approximate aggregate functions like approx_percentile_cont and approx_distinct, which uses HyperLogLog to produce very memory-efficient estimate distinct counts
  • Many new SQL window functions including rank, percent_rank, lag, and lead

Check out the Scalar, Aggregate,
and Window function docs for the complete list.

Arrays

Previously, Arroyo supported a limited set of operations on arrays, but only within certain contexts. Now we have complete array support, including a comprehensive set of array function, support for indexing via square brackets (like a[1]), support for serializing arrays as JSON and Avro, the ability to convert aggregates into arrays (via array_agg), and the ability to unroll arrays into separate rows via unnest.

For example, we can write unique IPs over a tumbling window as an array to Kafka with a query like this:

CREATE TABLE sink (
  values TEXT[]
) with (
  connector = 'confluent',
  connection_profile = 'confluent-working',
  format = 'avro',
  'avro.confluent_schema_registry' = 'true',
  topic = 'list_output',
  type = 'sink'
);

INSERT INTO sink
SELECT array_agg(DISTINCT ip) FROM logs
GROUP BY tumble(interval '10 seconds');

Source Fusion

Arroyo 0.10 includes new optimizations that reduce the amount of data read from a single source when it's used across multiple queries. Now, each source will only be read once rather than once per subquery.

This same optimization also applies to view, so compute can be reused across outputs.

Upgrading to 0.10

There are some backwards-incompatible changes in 0.10's SQL. Some pipelines written for 0.9 will need to be updated as described here.

Virtual fields

The syntax for virtual fields has changed to more closely match Postgres: we now require the STORED keyword. Previously:

event_time TIMESTAMP GENERATED ALWAYS AS (CAST(date as TIMESTAMP))

Now:

event_time TIMESTAMP GENERATED ALWAYS AS (CAST(date as TIMESTAMP)) STORED

UDFs

UDF definitions have changed slightly; now the UDF function must be annotated with an attribute macro #[arroyo_udf_plugin::udf], for example:

use arroyo_udf_plugin::udf;

#[udf]
fn plus_one(x: i64) -> i64 {
    x + 1
}

This change means we now support defining other ...

Read more

v0.9.1

02 Feb 21:33
Compare
Choose a tag to compare

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

  • Upgrade framer-motion to fix UDF popover by @jbeisen in #499
  • Fix cases where kafka client_config setting was not used by @mwylde in #520

Full Changelog: v0.9.0...v0.9.1