Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ The cookbook is a living document. :seedling:
6. [Deduplication](aggregations-and-analytics/06/06_dedup.md)
7. [Chained (Event) Time Windows](aggregations-and-analytics/07/07_chained_windows.md)
8. [Detecting Patterns with MATCH_RECOGNIZE](aggregations-and-analytics/08/08_match_recognize.md)
9. [Maintaining Materialized Views with Change Data Capture (CDC) and Debezium](aggregations-and-analytics/09/09_cdc_materialized_view.md)

### Other Built-in Functions & Operators
1. [Working with Dates and Timestamps](other-builtin-functions/01/01_date_time.md)
Expand Down
77 changes: 77 additions & 0 deletions aggregations-and-analytics/09/09_cdc_materialized_view.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
# 09 Maintaining Materialized Views with Change Data Capture (CDC) and Debezium

![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey)

:bulb: This example will show how you can use Flink SQL and Debezium to maintain a materialized view based on database changelog streams.

In the world of analytics, databases are still mostly seen as static sources of data — like a collection of business state(s) just sitting there, waiting to be queried. The reality is that most of the data stored in these databases is continuously produced and is continuously changing, so...why not _stream_ it?

Change Data Capture (CDC) allows you to do just that: track and propagate changes in a database based on its changelog (e.g. the [Write-Ahead-Log](https://www.postgresql.org/docs/current/wal-intro.html) in Postgres) to downstream consumers. [Debezium](https://debezium.io/) is a popular tool for CDC that Flink supports through **1)** the [Kafka SQL Connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/formats/debezium.html) and **2)** a set of "standalone" [Flink CDC Connectors](https://github.com/ververica/flink-cdc-connectors#flink-cdc-connectors).

#### Let's get to it!

In this example, you'll monitor a table with insurance claim data related to animal attacks in Australia, and use Flink SQL to maintain an aggregated **materialized view** that is **incrementally updated** with the latest claim costs. You can find a different version of this example deploying Debezium, Kafka and Kafka Connect in [this repository](https://github.com/morsapaes/flink-sql-CDC).

## Pre-requisites

You'll need a running Postgres service to follow this example, so we bundled everything up in a `docker-compose` script to keep it self-contained. The only pre-requisite is to have [Docker](https://docs.docker.com/get-docker/) installed on your machine. :whale:

To get the setup up and running, run:

`docker-compose build`

`docker-compose up -d`

Once all the services are up, you can start the Flink SQL client:

`docker-compose exec sql-client ./sql-client.sh`

## How it works

The source table is backed by the [`Flink CDC Postgres` connector](https://github.com/ververica/flink-cdc-connectors/wiki/Postgres-CDC-Connector), which reads the transaction log of the `postgres` database to continuously produce change events. So, whenever there is an `INSERT`, `UPDATE` or `DELETE` operation in the `claims.accident_claims` table, it will be propagated to Flink.

```sql
CREATE TABLE accident_claims (
claim_id INT,
claim_total FLOAT,
claim_total_receipt VARCHAR(50),
claim_currency VARCHAR(3),
member_id INT,
accident_date VARCHAR(20),
accident_type VARCHAR(20),
accident_detail VARCHAR(20),
claim_date VARCHAR(20),
claim_status VARCHAR(10),
ts_created VARCHAR(20),
ts_updated VARCHAR(20)
) WITH (
'connector' = 'postgres-cdc',
'hostname' = 'postgres',
'port' = '5432',
'username' = 'postgres',
'password' = 'postgres',
'database-name' = 'postgres',
'schema-name' = 'claims',
'table-name' = 'accident_claims'
);
```

After creating the changelog table, you can query it to find out the aggregated insurance costs of all cleared claims per animal type (`accident_detail`):

```sql
SELECT accident_detail,
SUM(claim_total) AS agg_claim_costs
FROM accident_claims
WHERE claim_status <> 'DENIED'
GROUP BY accident_detail;
```

How can you check that the CDC functionality is _actually_ working? The `docker` directory also includes a data generator script with a series of `INSERT` operations with new insurance claims (`postgres_datagen.sql`); if you run it, you can see how the query results update in (near) real-time:

`cat ./postgres_datagen.sql | docker exec -i flink-cdc-postgres psql -U postgres -d postgres`

In contrast to what would happen in a non-streaming SQL engine, using Flink SQL in combination with CDC allows you to get a consistent and continuous view of the state of the world, instead of a snapshot up to a specific point in time (i.e. the query's execution time).

## Example Output

![09_cdc_materialized_view](https://user-images.githubusercontent.com/23521087/109818653-81ee8180-7c33-11eb-9a76-b1004de8fe23.gif)
36 changes: 36 additions & 0 deletions aggregations-and-analytics/09/docker/client-image/Dockerfile
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
###############################################################################
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
###############################################################################

FROM flink:1.12.1-scala_2.11

# Copy sql-client script
COPY sql-client/ /opt/sql-client
RUN mkdir -p /opt/sql-client/lib

# Download connector libraries
RUN wget -P /opt/sql-client/lib/ https://repo1.maven.org/maven2/com/alibaba/ververica/flink-sql-connector-postgres-cdc/1.2.0/flink-sql-connector-postgres-cdc-1.2.0.jar; \
wget -P /opt/sql-client/lib/ https://repo1.maven.org/maven2/com/alibaba/ververica/flink-format-changelog-json/1.2.0/flink-format-changelog-json-1.2.0.jar;

# Copy configuration
COPY conf/* /opt/flink/conf/

WORKDIR /opt/sql-client
ENV SQL_CLIENT_HOME /opt/sql-client

COPY docker-entrypoint.sh /
ENTRYPOINT ["/docker-entrypoint.sh"]
Loading