Skip to content

Commit a0c07e9

Browse files
committed
[FLINK-19145][walkthroughs] Add PyFlink-walkthrough to Flink playground
1 parent 8b1cadb commit a0c07e9

15 files changed

+548
-0
lines changed

Diff for: README.md

+2
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,8 @@ Flink job. The playground is presented in detail in
1515

1616
* The **Table Walkthrough** (in the `table-walkthrough` folder) shows to use the Table API to build an analytics pipeline that reads streaming data from Kafka and writes results to MySQL, along with a real-time dashboard in Grafana. The walkthrough is presented in detail in ["Real Time Reporting with the Table API"](https://ci.apache.org/projects/flink/flink-docs-release-1.11/try-flink/table_api.html), which is part of the _Try Flink_ section of the Flink documentation.
1717

18+
* The **PyFlink Walkthrough** (in the `pyflink-walkthrough` folder) provides a complete example that uses the Python API, and guides you through the steps needed to run and manage Pyflink Jobs. The pipeline used in this walkthrough reads data from Kafka, performs aggregations, and writes results to Elasticsearch that are visualized with Kibana. This walkthrough is presented in detail in the [pyflink-walkthrough README](pyflink-walkthrough).
19+
1820
## About
1921

2022
Apache Flink is an open source project of The Apache Software Foundation (ASF).

Diff for: pyflink-walkthrough/Dockerfile

+48
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
###############################################################################
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing, software
13+
# distributed under the License is distributed on an "AS IS" BASIS,
14+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
# See the License for the specific language governing permissions and
16+
# limitations under the License.
17+
###############################################################################
18+
19+
###############################################################################
20+
# Build PyFlink Playground Image
21+
###############################################################################
22+
23+
FROM flink:1.11.0-scala_2.11
24+
ARG FLINK_VERSION=1.11.1
25+
26+
# Install pyflink
27+
RUN set -ex; \
28+
apt-get update; \
29+
apt-get -y install python3; \
30+
apt-get -y install python3-pip; \
31+
apt-get -y install python3-dev; \
32+
ln -s /usr/bin/python3 /usr/bin/python; \
33+
ln -s /usr/bin/pip3 /usr/bin/pip; \
34+
apt-get update; \
35+
python -m pip install --upgrade pip; \
36+
pip install apache-flink==1.11.1; \
37+
pip install kafka-python;
38+
39+
40+
# Download connector libraries
41+
RUN wget -P /opt/flink/lib/ https://repo.maven.apache.org/maven2/org/apache/flink/flink-json/${FLINK_VERSION}/flink-json-${FLINK_VERSION}.jar; \
42+
wget -P /opt/flink/lib/ https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-kafka_2.11/${FLINK_VERSION}/flink-sql-connector-kafka_2.11-${FLINK_VERSION}.jar; \
43+
wget -P /opt/flink/lib/ https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch7_2.11/${FLINK_VERSION}/flink-sql-connector-elasticsearch7_2.11-${FLINK_VERSION}.jar;
44+
45+
46+
RUN echo "taskmanager.memory.jvm-metaspace.size: 512m" >> /opt/flink/conf/flink-conf.yaml;
47+
48+
WORKDIR /opt/flink

Diff for: pyflink-walkthrough/README.md

+156
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,156 @@
1+
# pyflink-walkthrough
2+
3+
## Background
4+
5+
In this playground, you will learn how to build and run an end-to-end PyFlink pipeline for data analytics, covering the following steps:
6+
7+
* Reading data from a Kafka source;
8+
* Creating data using a [UDF](https://ci.apache.org/projects/flink/flink-docs-release-1.11/dev/python/table-api-users-guide/udfs/python_udfs.html);
9+
* Performing a simple aggregation over the source data;
10+
* Writing the results to Elasticsearch and visualizing them in Kibana.
11+
12+
The environment is based on Docker Compose, so the only requirement is that you have [Docker](https://docs.docker.com/get-docker/)
13+
installed in your machine.
14+
15+
### Kafka
16+
You will be using Kafka to store sample input data about payment transactions. A simple data generator [generate_source_data.py](generator/generate_source_data.py) is provided to
17+
continuously write new records to the `payment_msg` Kafka topic. Each record is structured as follows:
18+
19+
`{"createTime": "2020-08-12 06:29:02", "orderId": 1597213797, "payAmount": 28306.44976403719, "payPlatform": 0, "provinceId": 4}`
20+
21+
* `createTime`: The creation time of the transaction.
22+
* `orderId`: The id of the current transaction.
23+
* `payAmount`: The amount being paid with this transaction.
24+
* `payPlatform`: The platform used to create this payment: pc or mobile.
25+
* `provinceId`: The id of the province for the user.
26+
27+
### PyFlink
28+
29+
The transaction data will be processed with PyFlink using the Python script [payment_msg_processing.py](payment_msg_proccessing.py).
30+
This script will first map the `provinceId` in the input records to its corresponding province name using a Python UDF,
31+
and then compute the sum of the transaction amounts for each province. The following code snippet will explain the main processing logic in [payment_msg_processing.py](payment_msg_proccessing.py).
32+
33+
```python
34+
35+
t_env.from_path("payment_msg") \ # Get the created Kafka source table named payment_msg
36+
.select("province_id_to_name(provinceId) as province, payAmount") \ # Select the provinceId and payAmount field and transform the provinceId to province name by a UDF
37+
.group_by("province") \ # Group the selected fields by province
38+
.select("province, sum(payAmount) as pay_amount") \ # Sum up payAmount for each province
39+
.execute_insert("es_sink") # Write the result into ElaticSearch Sink
40+
41+
```
42+
43+
44+
### ElasticSearch
45+
46+
ElasticSearch is used to store the results and to provide an efficient query service.
47+
48+
### Kibana
49+
50+
Kibana is an open source data visualization dashboard for ElasticSearch. You will use it to visualize
51+
the total transaction paymentAmount and proportion for each provinces in this PyFlink pipeline through a dashboard.
52+
53+
## Setup
54+
55+
As mentioned, the environment for this walkthrough is based on Docker Compose; It uses a custom image
56+
to spin up Flink (JobManager+TaskManager), Kafka+Zookeeper, the data generator, and Elasticsearch+Kibana containers.
57+
58+
You can find the [docker-compose.yaml](docker-compose.yml) file of the pyflink-walkthrough in the `pyflink-walkthrough` root directory.
59+
60+
### Building the Docker image
61+
62+
First, build the Docker image by running:
63+
64+
```bash
65+
$ cd pyflink-walkthrough
66+
$ docker-compose build
67+
```
68+
69+
### Starting the Playground
70+
71+
Once the Docker image build is complete, run the following command to start the playground:
72+
73+
```bash
74+
$ docker-compose up -d
75+
```
76+
77+
One way of checking if the playground was successfully started is to access some of the services that are exposed:
78+
79+
1. visiting Flink Web UI [http://localhost:8081](http://localhost:8081).
80+
2. visiting Elasticsearch [http://localhost:9200](http://localhost:9200).
81+
3. visiting Kibana [http://localhost:5601](http://localhost:5601).
82+
83+
**Note:** you may need to wait around 1 minute before all the services come up.
84+
85+
### Checking the Kafka service
86+
87+
You can use the following command to read data from the Kafka topic and check whether it's generated correctly:
88+
```shell script
89+
$ docker-compose exec kafka kafka-console-consumer.sh --bootstrap-server kafka:9092 --topic payment_msg
90+
{"createTime":"2020-07-27 09:25:32.77","orderId":1595841867217,"payAmount":7732.44,"payPlatform":0,"provinceId":3}
91+
{"createTime":"2020-07-27 09:25:33.231","orderId":1595841867218,"payAmount":75774.05,"payPlatform":0,"provinceId":3}
92+
{"createTime":"2020-07-27 09:25:33.72","orderId":1595841867219,"payAmount":65908.55,"payPlatform":0,"provinceId":0}
93+
{"createTime":"2020-07-27 09:25:34.216","orderId":1595841867220,"payAmount":15341.11,"payPlatform":0,"provinceId":1}
94+
{"createTime":"2020-07-27 09:25:34.698","orderId":1595841867221,"payAmount":37504.42,"payPlatform":0,"provinceId":0}
95+
```
96+
You can also create a new topic by executing the following command:
97+
```shell script
98+
$ docker-compose exec kafka kafka-topics.sh --bootstrap-server kafka:9092 --create --topic <YOUR-TOPIC-NAME> --partitions 8 --replication-factor 1
99+
```
100+
101+
## Running the PyFlink job
102+
103+
1. Submit the PyFlink job.
104+
```shell script
105+
$ docker-compose exec jobmanager ./bin/flink run -py /opt/pyflink-walkthrough/payment_msg_proccessing.py -d
106+
```
107+
Navigate to the [Flink Web UI](http://localhost:8081) after the job is submitted successfully. There should be a job in the running job list.
108+
Click the job to get more details. You should see that the `StreamGraph` of the `payment_msg_proccessing` consists of two nodes, each with a parallelism of 1.
109+
There is also a table in the bottom of the page that shows some metrics for each node (e.g. bytes received/sent, records received/sent). Note that Flink's metrics only
110+
report bytes and records and records communicated within the Flink cluster, and so will always report 0 bytes and 0 records received by sources, and 0 bytes and 0 records
111+
sent to sinks - so don't be confused that noting is reported as being read from Kafka, or written to Elasticsearch.
112+
113+
![image](pic/submitted.png)
114+
115+
![image](pic/detail.png)
116+
117+
2. Navigate to the [Kibana UI](http://localhost:5601), open the menu list by clicking the menu button in the upper left corner, then choose the Dashboard item to turn to the dashboard page and choose the pre-created dashboard `payment_dashboard`.
118+
There will be a vertical bar chart and a pie chart demonstrating the total amount and the proportion of each province.
119+
120+
![image](pic/dash_board.png)
121+
122+
![image](pic/chart.png)
123+
124+
3. Stop the PyFlink job:
125+
126+
Visit the Flink Web UI at [http://localhost:8081/#/overview](http://localhost:8081/#/overview) , select the job, and click `Cancel Job` in the upper right corner.
127+
128+
![image](pic/cancel.png)
129+
130+
### Stopping the Playground
131+
132+
To stop the playground, run the following command:
133+
134+
```bash
135+
$ docker-compose down
136+
```
137+
138+
## Further Explorations
139+
140+
If you would like to explore this example more deeply, you can edit [payment_msg_processing.py](payment_msg_proccessing.py)
141+
or create new PyFlink projects that perform more complex processing. You can do this locally under
142+
the `pyflink-walkthrough` directory, since it is mounted on the `jobmanager` docker container.
143+
144+
Ideas:
145+
* Add your own Kafka source table;
146+
* Create a new index for the Elasticsearch sink;
147+
* Count the number of transactions, grouped by a 1 minute [tumbling window](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/tableApi.html#tumble-tumbling-windows) and `payPlatform`.
148+
149+
After making a modification, you can submit the new job by executing the same command mentioned at
150+
[Running the PyFlink Job](#running-the-pyflink-job)
151+
```shell script
152+
$ docker-compose exec jobmanager ./bin/flink run -py /opt/pyflink-walkthrough/payment_msg_proccessing.py -d
153+
```
154+
155+
Furthermore, you can also [create new Kibana dashboards](https://www.elastic.co/guide/en/kibana/7.8/dashboard-create-new-dashboard.html)
156+
to visualize other aspects of the data in Elasticsearch.

Diff for: pyflink-walkthrough/docker-compose.yml

+96
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,96 @@
1+
################################################################################
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing, software
13+
# distributed under the License is distributed on an "AS IS" BASIS,
14+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
# See the License for the specific language governing permissions and
16+
# limitations under the License.
17+
################################################################################
18+
19+
version: '2.1'
20+
services:
21+
jobmanager:
22+
build: .
23+
image: pyflink/pyflink:1.11.0-scala_2.11
24+
volumes:
25+
- .:/opt/pyflink-walkthrough
26+
hostname: "jobmanager"
27+
expose:
28+
- "6123"
29+
ports:
30+
- "8081:8081"
31+
command: jobmanager
32+
environment:
33+
- JOB_MANAGER_RPC_ADDRESS=jobmanager
34+
taskmanager:
35+
image: pyflink/pyflink:1.11.0-scala_2.11
36+
volumes:
37+
- .:/opt/pyflink-walkthrough
38+
expose:
39+
- "6121"
40+
- "6122"
41+
depends_on:
42+
- jobmanager
43+
command: taskmanager
44+
links:
45+
- jobmanager:jobmanager
46+
environment:
47+
- JOB_MANAGER_RPC_ADDRESS=jobmanager
48+
zookeeper:
49+
image: wurstmeister/zookeeper:3.4.6
50+
ports:
51+
- "2181:2181"
52+
kafka:
53+
image: wurstmeister/kafka:2.12-2.2.1
54+
ports:
55+
- "9092"
56+
depends_on:
57+
- zookeeper
58+
environment:
59+
HOSTNAME_COMMAND: "route -n | awk '/UG[ \t]/{print $$2}'"
60+
KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181
61+
KAFKA_CREATE_TOPICS: "payment_msg:1:1"
62+
volumes:
63+
- /var/run/docker.sock:/var/run/docker.sock
64+
generator:
65+
build: generator
66+
image: generator:1.0
67+
depends_on:
68+
- kafka
69+
elasticsearch:
70+
image: docker.elastic.co/elasticsearch/elasticsearch:7.8.0
71+
environment:
72+
- cluster.name=docker-cluster
73+
- bootstrap.memory_lock=true
74+
- "ES_JAVA_OPTS=-Xms512m -Xmx512m"
75+
- discovery.type=single-node
76+
ports:
77+
- "9200:9200"
78+
- "9300:9300"
79+
ulimits:
80+
memlock:
81+
soft: -1
82+
hard: -1
83+
nofile:
84+
soft: 65536
85+
hard: 65536
86+
kibana:
87+
image: docker.elastic.co/kibana/kibana:7.8.0
88+
ports:
89+
- "5601:5601"
90+
depends_on:
91+
- elasticsearch
92+
load-kibana-dashboard:
93+
build: ./kibana
94+
command: ['/bin/bash', '-c', 'cat /tmp/load/load_ndjson.sh | tr -d "\r" | bash']
95+
depends_on:
96+
- kibana

Diff for: pyflink-walkthrough/generator/Dockerfile

+27
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
###############################################################################
2+
# Licensed to the Apache Software Foundation (ASF) under one
3+
# or more contributor license agreements. See the NOTICE file
4+
# distributed with this work for additional information
5+
# regarding copyright ownership. The ASF licenses this file
6+
# to you under the Apache License, Version 2.0 (the
7+
# "License"); you may not use this file except in compliance
8+
# with the License. You may obtain a copy of the License at
9+
#
10+
# http://www.apache.org/licenses/LICENSE-2.0
11+
#
12+
# Unless required by applicable law or agreed to in writing, software
13+
# distributed under the License is distributed on an "AS IS" BASIS,
14+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
# See the License for the specific language governing permissions and
16+
# limitations under the License.
17+
###############################################################################
18+
19+
FROM python:3.7-alpine
20+
21+
RUN set -ex; \
22+
python -m pip install --upgrade pip; \
23+
pip install kafka-python;
24+
25+
ADD generate_source_data.py /
26+
27+
CMD ["python", "-u", "./generate_source_data.py"]

0 commit comments

Comments
 (0)