|
|
@@ -0,0 +1,306 @@ |
|
|
--- |
|
|
title: "Ibis goes real-time! Introducing the new Flink backend for Ibis" |
|
|
author: "Deepyaman Datta" |
|
|
date: "2024-02-12" |
|
|
categories: |
|
|
- blog |
|
|
- flink |
|
|
- stream processing |
|
|
--- |
|
|
|
|
|
## Introduction |
|
|
|
|
|
Ibis 8.0 marks the official release of the Apache Flink backend for Ibis. Flink |
|
|
is one of the most established stream-processing frameworks out there and a |
|
|
central part of the real-time data infrastructure at companies like DoorDash, |
|
|
LinkedIn, Netflix, and Uber. It is commonly applied in use cases such as fraud |
|
|
detection, anomaly detection, real-time recommendation, dynamic pricing, and |
|
|
online advertising. The Flink backend is also the first streaming backend Ibis |
|
|
supports. Follow along as we define and execute a simple streaming job using |
|
|
Ibis! |
|
|
|
|
|
## Installation prerequisites |
|
|
|
|
|
* **Docker Compose:** This tutorial uses Docker Compose to manage an Apache |
|
|
Kafka environment (including sample data generation) and a Flink cluster (for |
|
|
[remote execution](#remote-execution)). You can [download and install Docker |
|
|
Compose from the official website](https://docs.docker.com/compose/install/). |
|
|
* **JDK 11:** Flink requires Java 11. If you don't already have JDK 11 |
|
|
installed, you can [get the appropriate Eclipse Temurin |
|
|
release](https://adoptium.net/temurin/releases/?package=jdk&version=11). |
|
|
* **Python:** To follow along, you need Python 3.9 or 3.10. |
|
|
|
|
|
## Installing the Flink backend for Ibis |
|
|
|
|
|
We use a Python client to explore data in Kafka topics. You can install it, |
|
|
alongside the Flink backend for Ibis, with `pip`, `conda`, `mamba`, or `pixi`: |
|
|
|
|
|
::: {.panel-tabset} |
|
|
|
|
|
## Using `pip` |
|
|
|
|
|
```bash |
|
|
pip install ibis-framework apache-flink kafka-python |
|
|
``` |
|
|
|
|
|
## Using `conda` |
|
|
|
|
|
```bash |
|
|
conda install -c conda-forge ibis-flink |
|
|
``` |
|
|
|
|
|
## Using `mamba` |
|
|
|
|
|
```bash |
|
|
mamba install -c conda-forge ibis-flink |
|
|
``` |
|
|
|
|
|
## Using `pixi` |
|
|
|
|
|
```bash |
|
|
pixi add ibis-flink |
|
|
``` |
|
|
|
|
|
::: |
|
|
|
|
|
## Spinning up the services using Docker Compose |
|
|
|
|
|
The [ibis-project/ibis-flink-example GitHub |
|
|
repository](https://github.com/ibis-project/ibis-flink-example) includes the |
|
|
relevant Docker Compose configuration for this tutorial. Clone the repository, |
|
|
and run `docker compose up` from the cloned directory to create Kafka topics, |
|
|
generate sample data, and launch a Flink cluster: |
|
|
|
|
|
```bash |
|
|
git clone https://github.com/claypotai/ibis-flink-example.git |
|
|
cd ibis-flink-example |
|
|
docker compose up |
|
|
``` |
|
|
|
|
|
::: {.callout-tip} |
|
|
If you don't intend to try [remote execution](#remote-execution), you can start |
|
|
only the Kafka-related services with `docker compose up kafka init-kafka |
|
|
data-generator`. |
|
|
::: |
|
|
|
|
|
After a few seconds, you should see messages indicating your Kafka environment |
|
|
is ready: |
|
|
|
|
|
```bash |
|
|
ibis-flink-example-init-kafka-1 | Successfully created the following topics: |
|
|
ibis-flink-example-init-kafka-1 | payment_msg |
|
|
ibis-flink-example-init-kafka-1 | sink |
|
|
ibis-flink-example-init-kafka-1 exited with code 0 |
|
|
ibis-flink-example-data-generator-1 | Connected to Kafka |
|
|
ibis-flink-example-data-generator-1 | Producing 20000 records to Kafka topic payment_msg |
|
|
``` |
|
|
|
|
|
This example uses mock payments data. The `payment_msg` Kafka topic contains |
|
|
messages in the following format: |
|
|
|
|
|
```json |
|
|
{ |
|
|
"createTime": "2023-09-20 22:19:02.224", |
|
|
"orderId": 1695248388, |
|
|
"payAmount": 88694.71922270155, |
|
|
"payPlatform": 0, |
|
|
"provinceId": 6 |
|
|
} |
|
|
``` |
|
|
|
|
|
In a separate terminal, we can explore what these messages look like: |
|
|
|
|
|
```{python} |
|
|
from itertools import islice |
|
|
|
|
|
from kafka import KafkaConsumer |
|
|
|
|
|
consumer = KafkaConsumer("payment_msg") |
|
|
for msg in islice(consumer, 3): |
|
|
print(msg) |
|
|
``` |
|
|
|
|
|
## Running the tutorial |
|
|
|
|
|
This tutorial uses Ibis with the Flink backend to process the aforementioned |
|
|
payment messages. You can choose to either [run it locally](#local-execution) or |
|
|
[submit a job to an already-running Flink cluster](#remote-execution). |
|
|
|
|
|
### Local execution |
|
|
|
|
|
The simpler option is to run the example using the Flink mini cluster. |
|
|
|
|
|
#### Create a table environment |
|
|
|
|
|
The [table |
|
|
environment](https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/python/table/table_environment/) |
|
|
serves as the main entry point for interacting with the Flink runtime. The |
|
|
`flink` backend does not create `TableEnvironment` objects; you must create a |
|
|
`TableEnvironment` and pass that to |
|
|
[`ibis.flink.connect`](../../backends/flink.qmd#ibis.flink.connect): |
|
|
|
|
|
```{python} |
|
|
import ibis |
|
|
from pyflink.table import EnvironmentSettings, TableEnvironment |
|
|
|
|
|
env_settings = EnvironmentSettings.in_streaming_mode() |
|
|
table_env = TableEnvironment.create(env_settings) |
|
|
table_env.get_config().set("parallelism.default", "1") # <1> |
|
|
|
|
|
con = ibis.flink.connect(table_env) |
|
|
``` |
|
|
1. write all the data to one file |
|
|
|
|
|
Flink’s streaming connectors aren't part of the binary distribution. Link the |
|
|
[Kafka |
|
|
connector](https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/connectors/table/kafka/) |
|
|
for cluster execution by adding the JAR file from the cloned repository. [Ibis |
|
|
exposes the `raw_sql` method for situations like this, where you need to run |
|
|
arbitrary SQL that cannot be modeled as a table |
|
|
expression](https://ibis-project.org/how-to/extending/sql#backend.raw_sql): |
|
|
|
|
|
```{python} |
|
|
#| echo: false |
|
|
#| output: false |
|
|
|
|
|
import urllib.request |
|
|
|
|
|
urllib.request.urlretrieve( |
|
|
"https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka/3.0.2-1.18/flink-sql-connector-kafka-3.0.2-1.18.jar", |
|
|
filename="flink-sql-connector-kafka-3.0.2-1.18.jar", |
|
|
) |
|
|
``` |
|
|
|
|
|
```{python} |
|
|
#| output: false |
|
|
|
|
|
con.raw_sql("ADD JAR 'flink-sql-connector-kafka-3.0.2-1.18.jar'") |
|
|
``` |
|
|
|
|
|
#### Create the source and sink tables |
|
|
|
|
|
Use |
|
|
[`create_table`](../../backends/flink.qmd#ibis.backends.flink.Backend.create_table) |
|
|
to register tables. Notice the new top-level `ibis.watermark` API for |
|
|
[specifying a watermark |
|
|
strategy](https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/concepts/time/#event-time-and-watermarks). |
|
|
|
|
|
```{python} |
|
|
source_schema = ibis.schema( # <1> |
|
|
{ # <1> |
|
|
"createTime": "timestamp(3)", # <1> |
|
|
"orderId": "int64", # <1> |
|
|
"payAmount": "float64", # <1> |
|
|
"payPlatform": "int32", # <1> |
|
|
"provinceId": "int32", # <1> |
|
|
} # <1> |
|
|
) # <1> |
|
|
|
|
|
source_configs = { # <1> |
|
|
"connector": "kafka", # <1> |
|
|
"topic": "payment_msg", # <1> |
|
|
"properties.bootstrap.servers": "localhost:9092", # <1> |
|
|
"properties.group.id": "test_3", # <1> |
|
|
"scan.startup.mode": "earliest-offset", # <1> |
|
|
"format": "json", # <1> |
|
|
} # <1> |
|
|
|
|
|
t = con.create_table( # <1> |
|
|
"payment_msg", # <1> |
|
|
schema=source_schema, # <1> |
|
|
tbl_properties=source_configs, # <1> |
|
|
watermark=ibis.watermark( # <1> |
|
|
time_col="createTime", allowed_delay=ibis.interval(seconds=15) # <1> |
|
|
), # <1> |
|
|
) # <1> |
|
|
|
|
|
sink_schema = ibis.schema( # <2> |
|
|
{ # <2> |
|
|
"province_id": "int32", # <2> |
|
|
"pay_amount": "float64", # <2> |
|
|
} # <2> |
|
|
) # <2> |
|
|
|
|
|
sink_configs = { # <2> |
|
|
"connector": "kafka", # <3> |
|
|
"topic": "sink", # <2> |
|
|
"properties.bootstrap.servers": "localhost:9092", # <2> |
|
|
"format": "json", # <2> |
|
|
} # <2> |
|
|
|
|
|
con.create_table( # <2> |
|
|
"total_amount_by_province_id", schema=sink_schema, tbl_properties=sink_configs # <2> |
|
|
) # <2> |
|
|
``` |
|
|
1. create source Table |
|
|
2. create sink Table |
|
|
|
|
|
#### Perform calculations |
|
|
|
|
|
Compute the total pay amount per province in the past 10 seconds (as of each |
|
|
message, for the province in the incoming message): |
|
|
|
|
|
```{python} |
|
|
agged = t.select( |
|
|
province_id=t.provinceId, |
|
|
pay_amount=t.payAmount.sum().over( |
|
|
range=(-ibis.interval(seconds=10), 0), |
|
|
group_by=t.provinceId, |
|
|
order_by=t.createTime, |
|
|
), |
|
|
) |
|
|
``` |
|
|
|
|
|
Finally, emit the query result to the sink table: |
|
|
|
|
|
```{python} |
|
|
con.insert("total_amount_by_province_id", agged) |
|
|
``` |
|
|
|
|
|
### Remote execution |
|
|
|
|
|
You can also submit the example to the [remote cluster started using Docker |
|
|
Compose](#spinning-up-the-services-using-docker-compose). The |
|
|
`window_aggregation.py` file in the cloned repository contains the [same steps |
|
|
that we performed for local execution](#local-execution). We will [use the |
|
|
method described in the official Flink |
|
|
documentation](https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/cli/#submitting-pyflink-jobs). |
|
|
|
|
|
::: {.callout-tip} |
|
|
You can find the `./bin/flink` executable with the following command: |
|
|
|
|
|
```bash |
|
|
python -c'from pathlib import Path; import pyflink; print(Path(pyflink.__spec__.origin).parent / "bin" / "flink")' |
|
|
``` |
|
|
::: |
|
|
|
|
|
My full command looks like this: |
|
|
|
|
|
```bash |
|
|
/opt/miniconda3/envs/ibis-dev/lib/python3.10/site-packages/pyflink/bin/flink run --jobmanager localhost:8081 --python window_aggregation.py |
|
|
``` |
|
|
|
|
|
The command will exit after displaying a submission message: |
|
|
|
|
|
``` |
|
|
Job has been submitted with JobID b816faaf5ef9126ea5b9b6a37012cf56 |
|
|
``` |
|
|
|
|
|
## Viewing the results |
|
|
|
|
|
Similar to how we viewed messages in the `payment_msg` topic, we can print |
|
|
results from the `sink` topic: |
|
|
|
|
|
```{python} |
|
|
consumer = KafkaConsumer("sink") |
|
|
for msg in islice(consumer, 10): |
|
|
print(msg) |
|
|
``` |
|
|
|
|
|
VoilĂ ! You've run your first streaming application using Ibis. |
|
|
|
|
|
## Shutting down the Compose environment |
|
|
|
|
|
Press <kbd>Ctrl</kbd>+<kbd>C</kbd> to stop the Docker Compose containers. Once |
|
|
stopped, run `docker compose down` to remove the services created for this |
|
|
tutorial. |