├── .gitignore ├── aggregations-and-analytics ├── 09_cdc_materialized_view │ ├── docker │ │ ├── client-image │ │ │ ├── docker-entrypoint.sh │ │ │ ├── sql-client │ │ │ │ └── sql-client.sh │ │ │ ├── conf │ │ │ │ ├── flink-conf.yaml │ │ │ │ └── sql-client-conf.yaml │ │ │ ├── Dockerfile │ │ │ └── LICENSE │ │ ├── postgres-image │ │ │ └── Dockerfile │ │ └── docker-compose.yml │ └── 09_cdc_materialized_view.md ├── 12_lag │ ├── 12_lag.gif │ └── 12_lag.md ├── 11_window_top_n │ ├── 11_window_top_n.png │ └── 11_window_top_n.md ├── 01_group_by_window │ ├── 01_group_by_window_tvf_result.png │ ├── 01_group_by_window.md │ └── 01_group_by_window_tvf.md ├── 10_hopping_time_windows │ ├── 10_hopping_time_windows.png │ └── 10_hopping_time_windows.md ├── 06_dedup │ └── 06_dedup.md ├── 05_top_n │ └── 05_top_n.md ├── 04_over │ └── 04_over.md ├── 02_watermarks │ └── 02_watermarks.md ├── 03_group_by_session_window │ └── 03_group_by_session_window.md ├── 08_match_recognize │ └── 08_match_recognize.md └── 07_chained_windows │ └── 07_chained_windows.md ├── foundations ├── 09_convert_timezones │ ├── 09_convert_timezones.gif │ └── 09_convert_timezones.md ├── 01_create_table │ └── 01_create_table.md ├── 04_where │ └── 04_where.md ├── 07_views │ └── 07_views.md ├── 05_group_by │ └── 05_group_by.md ├── 03_temporary_table │ └── 03_temporary_table.md ├── 02_insert_into │ └── 02_insert_into.md ├── 06_order_by │ └── 06_order_by.md └── 08_statement_sets │ └── 08_statement_sets.md ├── other-builtin-functions ├── 03_current_watermark │ ├── 03_late_data.png │ ├── 03_current_watermark.png │ └── 03_current_watermark.md ├── 05_expanding_arrays │ ├── 05_unnested_data.png │ ├── 05_complex_data_types.png │ ├── 05_joined_unnested_data.png │ └── 05_expanding_arrays.md ├── 06_split_strings_into_maps │ ├── 06_split_strings_into_maps.png │ └── 06_split_strings_into_maps.md ├── 04_override_table_options │ ├── 04_override_table_options.screen01.png │ ├── 04_override_table_options.screen02.png │ ├── 04_override_table_options.md │ └── airports.csv ├── 02_union-all │ └── 02_union-all.md └── 01_date_time │ └── 01_date_time.md ├── udfs └── 01_python_udfs │ ├── python_udf.py │ └── 01_python_udfs.md ├── joins ├── 01_regular_joins │ └── 01_regular_joins.md ├── 02_interval_joins │ └── 02_interval_joins.md ├── 06_lateral_join │ └── 06_lateral_join.md ├── 04_lookup_joins │ └── 04_lookup_joins.md ├── 03_kafka_join │ └── 03_kafka_join.md └── 05_star_schema │ └── 05_star_schema.md ├── README.md └── LICENSE /.gitignore: -------------------------------------------------------------------------------- 1 | .idea -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/docker-entrypoint.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | tail -f /dev/null -------------------------------------------------------------------------------- /aggregations-and-analytics/12_lag/12_lag.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/aggregations-and-analytics/12_lag/12_lag.gif -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/postgres-image/Dockerfile: -------------------------------------------------------------------------------- 1 | FROM debezium/postgres:11 2 | 3 | COPY postgres_bootstrap.sql /docker-entrypoint-initdb.d/ -------------------------------------------------------------------------------- /foundations/09_convert_timezones/09_convert_timezones.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/foundations/09_convert_timezones/09_convert_timezones.gif -------------------------------------------------------------------------------- /aggregations-and-analytics/11_window_top_n/11_window_top_n.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/aggregations-and-analytics/11_window_top_n/11_window_top_n.png -------------------------------------------------------------------------------- /other-builtin-functions/03_current_watermark/03_late_data.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/03_current_watermark/03_late_data.png -------------------------------------------------------------------------------- /other-builtin-functions/05_expanding_arrays/05_unnested_data.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/05_expanding_arrays/05_unnested_data.png -------------------------------------------------------------------------------- /other-builtin-functions/03_current_watermark/03_current_watermark.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/03_current_watermark/03_current_watermark.png -------------------------------------------------------------------------------- /other-builtin-functions/05_expanding_arrays/05_complex_data_types.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/05_expanding_arrays/05_complex_data_types.png -------------------------------------------------------------------------------- /other-builtin-functions/05_expanding_arrays/05_joined_unnested_data.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/05_expanding_arrays/05_joined_unnested_data.png -------------------------------------------------------------------------------- /aggregations-and-analytics/01_group_by_window/01_group_by_window_tvf_result.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/aggregations-and-analytics/01_group_by_window/01_group_by_window_tvf_result.png -------------------------------------------------------------------------------- /aggregations-and-analytics/10_hopping_time_windows/10_hopping_time_windows.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/aggregations-and-analytics/10_hopping_time_windows/10_hopping_time_windows.png -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/sql-client/sql-client.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | ${FLINK_HOME}/bin/sql-client.sh embedded -d ${FLINK_HOME}/conf/sql-client-conf.yaml -l ${SQL_CLIENT_HOME}/lib -------------------------------------------------------------------------------- /other-builtin-functions/06_split_strings_into_maps/06_split_strings_into_maps.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/06_split_strings_into_maps/06_split_strings_into_maps.png -------------------------------------------------------------------------------- /other-builtin-functions/04_override_table_options/04_override_table_options.screen01.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/04_override_table_options/04_override_table_options.screen01.png -------------------------------------------------------------------------------- /other-builtin-functions/04_override_table_options/04_override_table_options.screen02.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ververica/flink-sql-cookbook/HEAD/other-builtin-functions/04_override_table_options/04_override_table_options.screen02.png -------------------------------------------------------------------------------- /udfs/01_python_udfs/python_udf.py: -------------------------------------------------------------------------------- 1 | from pyflink.table import DataTypes 2 | from pyflink.table.udf import udf 3 | 4 | us_cities = {"Chicago","Portland","Seattle","New York"} 5 | 6 | @udf(input_types=[DataTypes.STRING(), DataTypes.FLOAT()], 7 | result_type=DataTypes.FLOAT()) 8 | def to_fahr(city, temperature): 9 | 10 | if city in us_cities: 11 | 12 | fahr = ((temperature * 9.0 / 5.0) + 32.0) 13 | 14 | return fahr 15 | else: 16 | return temperature 17 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/conf/flink-conf.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Copyright 2019 Ververica GmbH 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); 5 | # you may not use this file except in compliance with the License. 6 | # You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | ################################################################################ 16 | 17 | jobmanager.rpc.address: jobmanager 18 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/docker-compose.yml: -------------------------------------------------------------------------------- 1 | version: '3' 2 | services: 3 | postgres: 4 | container_name: flink-cdc-postgres 5 | build: ./postgres-image 6 | image: demo/postgres-pop 7 | ports: 8 | - 5432:5432 9 | environment: 10 | - POSTGRES_USER=postgres 11 | - POSTGRES_PASSWORD=postgres 12 | volumes: 13 | - ${PWD}/postgres-image:/docker-entrypoint-initdb.d 14 | sql-client: 15 | container_name: flink-cdc-sql-client 16 | build: ./client-image 17 | image: demo/debezium-flink-sql 18 | depends_on: 19 | - jobmanager 20 | environment: 21 | FLINK_JOBMANAGER_HOST: jobmanager 22 | ZOOKEEPER_CONNECT: zookeeper 23 | KAFKA_BOOTSTRAP: kafka 24 | ES_HOST: elasticsearch 25 | jobmanager: 26 | container_name: flink-cdc-jobmanager 27 | image: flink:1.12.1-scala_2.11 28 | ports: 29 | - "8081:8081" 30 | command: jobmanager 31 | environment: 32 | - JOB_MANAGER_RPC_ADDRESS=jobmanager 33 | taskmanager: 34 | container_name: flink-cdc-taskmanager 35 | image: flink:1.12.1-scala_2.11 36 | depends_on: 37 | - jobmanager 38 | command: taskmanager 39 | environment: 40 | - JOB_MANAGER_RPC_ADDRESS=jobmanager -------------------------------------------------------------------------------- /foundations/01_create_table/01_create_table.md: -------------------------------------------------------------------------------- 1 | # 01 Creating Tables 2 | 3 | > :bulb: This example will show how to create a table using SQL DDL. 4 | 5 | Flink SQL operates against logical tables, just like a traditional database. 6 | However, it does not maintain tables internally but always operates against external systems. 7 | 8 | Table definitions are in two parts; the logical schema and connector configuration. The logical schema defines the columns and types in the table and is what queries operate against. 9 | The connector configuration is contained in the `WITH` clause and defines the physical system that backs this table. 10 | This example uses the `datagen` connector which generates rows in memory and is convenient for testing queries. 11 | 12 | You can test the table is properly created by running a simple `SELECT` statement. 13 | In Ververica Platform you will see the results printed to the UI in the query preview. 14 | 15 | ## Script 16 | 17 | ```sql 18 | 19 | CREATE TABLE orders ( 20 | order_uid BIGINT, 21 | product_id BIGINT, 22 | price DECIMAL(32, 2), 23 | order_time TIMESTAMP(3) 24 | ) WITH ( 25 | 'connector' = 'datagen' 26 | ); 27 | 28 | SELECT * FROM orders; 29 | ``` 30 | 31 | ## Example Output 32 | 33 | ![01_create_table](https://user-images.githubusercontent.com/23521087/105504017-913eee80-5cc7-11eb-868c-7b78b1b95b71.png) 34 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/Dockerfile: -------------------------------------------------------------------------------- 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 flink:1.12.1-scala_2.11 20 | 21 | # Copy sql-client script 22 | COPY sql-client/ /opt/sql-client 23 | RUN mkdir -p /opt/sql-client/lib 24 | 25 | # Download connector libraries 26 | 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; \ 27 | 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; 28 | 29 | # Copy configuration 30 | COPY conf/* /opt/flink/conf/ 31 | 32 | WORKDIR /opt/sql-client 33 | ENV SQL_CLIENT_HOME /opt/sql-client 34 | 35 | COPY docker-entrypoint.sh / 36 | ENTRYPOINT ["/docker-entrypoint.sh"] 37 | -------------------------------------------------------------------------------- /foundations/04_where/04_where.md: -------------------------------------------------------------------------------- 1 | # 04 Filtering Data 2 | 3 | > :bulb: This example will show how to filter server logs in real-time using a standard `WHERE` clause. 4 | 5 | The table it uses, `server_logs`, is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker) which continuously generates rows in memory based on Java Faker expressions and is convenient for testing queries. 6 | As such, it is an alternative to the built-in `datagen` connector used for example in [the first recipe](../01_create_table/01_create_table.md). 7 | 8 | You can continuously filter these logs for those requests that experience authx issues with a simple `SELECT` statement with a `WHERE` clause filtering on the auth related HTTP status codes. 9 | In Ververica Platform you will see the results printed to the UI in the query preview. 10 | 11 | ## Script 12 | 13 | ```sql 14 | CREATE TABLE server_logs ( 15 | client_ip STRING, 16 | client_identity STRING, 17 | userid STRING, 18 | log_time TIMESTAMP(3), 19 | request_line STRING, 20 | status_code STRING, 21 | size INT 22 | ) WITH ( 23 | 'connector' = 'faker', 24 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 25 | 'fields.client_identity.expression' = '-', 26 | 'fields.userid.expression' = '-', 27 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 28 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 29 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 30 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 31 | ); 32 | 33 | SELECT 34 | log_time, 35 | request_line, 36 | status_code 37 | FROM server_logs 38 | WHERE 39 | status_code IN ('403', '401'); 40 | ``` 41 | 42 | ## Example Output 43 | 44 | ![04_where](https://user-images.githubusercontent.com/23521087/105504095-a6b41880-5cc7-11eb-9606-978e86add144.png) 45 | -------------------------------------------------------------------------------- /foundations/09_convert_timezones/09_convert_timezones.md: -------------------------------------------------------------------------------- 1 | # 09 Convert timestamps with timezones 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.19%2B-lightgrey) 4 | 5 | > :bulb: In this recipe, you will learn how to consolidate timestamps with different time zones to UTC. 6 | 7 | Timestamps in incoming data can refer to different time zones and consolidating them to the same time zone (e.g. UTC) is a prerequisite to ensure correctness in temporal analysis. 8 | 9 | The source table (`iot_status`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates fake IoT status messages in memory based on Java Faker expressions. 10 | 11 | In this recipe we create a table which contains IoT devices status updates including timestamp and device time zone, which we'll convert to UTC. 12 | 13 | We create the table first, then use the [`CONVERT_TZ`](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/functions/systemfunctions/#temporal-functions) function to convert the timestamp to UTC. The `CONVERT_TZ` function requires the input timestamp to be passed as string, thus we apply the cast function to `iot_timestamp`. 14 | 15 | ```sql 16 | CREATE TABLE iot_status ( 17 | device_ip STRING, 18 | device_timezone STRING, 19 | iot_timestamp TIMESTAMP(3), 20 | status_code STRING 21 | ) WITH ( 22 | 'connector' = 'faker', 23 | 'fields.device_ip.expression' = '#{Internet.publicIpV4Address}', 24 | 'fields.device_timezone.expression' = '#{regexify ''(America\/Los_Angeles|Europe\/Rome|Europe\/London|Australia\/Sydney){1}''}', 25 | 'fields.iot_timestamp.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 26 | 'fields.status_code.expression' = '#{regexify ''(OK|KO|WARNING){1}''}', 27 | 'rows-per-second' = '3' 28 | ); 29 | 30 | SELECT 31 | device_ip, 32 | device_timezone, 33 | iot_timestamp, 34 | convert_tz(cast(iot_timestamp as string), device_timezone, 'UTC') iot_timestamp_utc, 35 | status_code 36 | FROM iot_status; 37 | ``` 38 | 39 | The 40 | 41 | ## Example Output 42 | 43 | ![09_convert_timezones](09_convert_timezones.gif) 44 | -------------------------------------------------------------------------------- /aggregations-and-analytics/12_lag/12_lag.md: -------------------------------------------------------------------------------- 1 | # 12 Retrieve previous row value without self-join 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.19%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to retrieve the previous value and compute trends for a specific data partition. 6 | 7 | The source table (`fake_stocks`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates fake stock quotation in memory based on Java Faker expressions. 8 | 9 | In this recipe we're going to create a table which contains stock ticker updates for which we want to determine if the new stock price has gone up or down compared to its previous value. 10 | 11 | First we create the table, then use a select statement including the [LAG](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/functions/systemfunctions/#aggregate-functions) function to retrieve the previous stock value. Finally using the `case` statement in the final select we compare the current stock price against the previous value to determine the trend. 12 | 13 | ```sql 14 | CREATE TABLE fake_stocks ( 15 | stock_name STRING, 16 | stock_value double, 17 | log_time AS PROCTIME() 18 | ) WITH ( 19 | 'connector' = 'faker', 20 | 'fields.stock_name.expression' = '#{regexify ''(Deja\ Brew|Jurassic\ Pork|Lawn\ \&\ Order|Pita\ Pan|Bread\ Pitt|Indiana\ Jeans|Thai\ Tanic){1}''}', 21 | 'fields.stock_value.expression' = '#{number.randomDouble ''2'',''10'',''20''}', 22 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 23 | 'rows-per-second' = '10' 24 | ); 25 | 26 | WITH current_and_previous as ( 27 | select 28 | stock_name, 29 | log_time, 30 | stock_value, 31 | lag(stock_value, 1) over (partition by stock_name order by log_time) previous_value 32 | from fake_stocks 33 | ) 34 | select *, 35 | case 36 | when stock_value > previous_value then '▲' 37 | when stock_value < previous_value then '▼' 38 | else '=' 39 | end as trend 40 | from current_and_previous; 41 | ``` 42 | 43 | ## Example Output 44 | 45 | ![12_lag](12_lag.gif) 46 | -------------------------------------------------------------------------------- /aggregations-and-analytics/06_dedup/06_dedup.md: -------------------------------------------------------------------------------- 1 | # 06 Deduplication 2 | 3 | > :bulb: This example will show how you can identify and filter out duplicates in a stream of events. 4 | 5 | There are different ways that duplicate events can end up in your data sources, from human error to application bugs. Regardless of the origin, unclean data can have a real impact in the quality (and correctness) of your results. Suppose that your order system occasionally generates duplicate events with the same `order_id`, and that you're only interested in keeping the most recent event for downstream processing. 6 | 7 | As a first step, you can use a combination of the `COUNT` function and the `HAVING` clause to check if and which orders have more than one event; and then filter out these events using `ROW_NUMBER()`. In practice, deduplication is a special case of [Top-N aggregation](../05_top_n/05_top_n.md), where N is 1 (`rownum = 1`) and the ordering column is either the processing or event time of events. 8 | 9 | ## Script 10 | 11 | The source table `orders` is backed by the built-in [`datagen` connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/datagen.html), which continuously generates rows in memory. 12 | 13 | ```sql 14 | CREATE TABLE orders ( 15 | id INT, 16 | order_time AS CURRENT_TIMESTAMP, 17 | WATERMARK FOR order_time AS order_time - INTERVAL '5' SECONDS 18 | ) 19 | WITH ( 20 | 'connector' = 'datagen', 21 | 'rows-per-second'='10', 22 | 'fields.id.kind'='random', 23 | 'fields.id.min'='1', 24 | 'fields.id.max'='100' 25 | ); 26 | 27 | --Check for duplicates in the `orders` table 28 | SELECT id AS order_id, 29 | COUNT(*) AS order_cnt 30 | FROM orders o 31 | GROUP BY id 32 | HAVING COUNT(*) > 1; 33 | 34 | --Use deduplication to keep only the latest record for each `order_id` 35 | SELECT 36 | order_id, 37 | order_time 38 | FROM ( 39 | SELECT id AS order_id, 40 | order_time, 41 | ROW_NUMBER() OVER (PARTITION BY id ORDER BY order_time) AS rownum 42 | FROM orders 43 | ) 44 | WHERE rownum = 1; 45 | ``` 46 | 47 | ## Example Output 48 | 49 | ![20_dedup](https://user-images.githubusercontent.com/23521087/102718503-b87d5700-42e8-11eb-8b45-4f9908e8e14e.gif) 50 | -------------------------------------------------------------------------------- /aggregations-and-analytics/11_window_top_n/11_window_top_n.md: -------------------------------------------------------------------------------- 1 | # 11 Window Top-N 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.13%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to calculate the Top 3 suppliers who have the highest sales for every tumbling 5 minutes window. 6 | 7 | The source table (`orders`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | In our previous recipes we've shown how you can [aggregate time series data](../01_group_by_window/01_group_by_window_tvf.md) using the `TUMBLE` function and also how you can get continuous [Top-N results](../05_top_n/05_top_n.md). 10 | In this recipe, you will use the `Window Top-N` [feature](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/sql/queries/window-topn/) to display the top 3 suppliers with the highest sales every 5 minutes. 11 | 12 | The difference between the regular Top-N and this Window Top-N, is that Window Top-N only emits final results, which is the total top N records at the end of the window. 13 | 14 | ```sql 15 | CREATE TABLE orders ( 16 | bidtime TIMESTAMP(3), 17 | price DOUBLE, 18 | item STRING, 19 | supplier STRING, 20 | WATERMARK FOR bidtime AS bidtime - INTERVAL '5' SECONDS 21 | ) WITH ( 22 | 'connector' = 'faker', 23 | 'fields.bidtime.expression' = '#{date.past ''30'',''SECONDS''}', 24 | 'fields.price.expression' = '#{Number.randomDouble ''2'',''1'',''150''}', 25 | 'fields.item.expression' = '#{Commerce.productName}', 26 | 'fields.supplier.expression' = '#{regexify ''(Alice|Bob|Carol|Alex|Joe|James|Jane|Jack)''}', 27 | 'rows-per-second' = '100' 28 | ); 29 | 30 | SELECT * 31 | FROM ( 32 | SELECT *, ROW_NUMBER() OVER (PARTITION BY window_start, window_end ORDER BY price DESC) as rownum 33 | FROM ( 34 | SELECT window_start, window_end, supplier, SUM(price) as price, COUNT(*) as cnt 35 | FROM TABLE( 36 | TUMBLE(TABLE orders, DESCRIPTOR(bidtime), INTERVAL '10' MINUTES)) 37 | GROUP BY window_start, window_end, supplier 38 | ) 39 | ) WHERE rownum <= 3; 40 | ``` 41 | 42 | ## Example Output 43 | 44 | ![11_window_top_n](11_window_top_n.png) 45 | -------------------------------------------------------------------------------- /aggregations-and-analytics/10_hopping_time_windows/10_hopping_time_windows.md: -------------------------------------------------------------------------------- 1 | # 10 Hopping Time Windows 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.13%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to calculate a moving average in real-time using a `HOP` window. 6 | 7 | The source table (`bids`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | In one of our previous recipes we've shown how you can [aggregate time series data](../01_group_by_window/01_group_by_window_tvf.md) using `TUMBLE`. 10 | To display every 30 seconds the moving average of bidding prices per currency per 1 minute, we will use the built-in `HOP` [function](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/sql/queries/window-agg/). 11 | 12 | The difference between a `HOP` and a `TUMBLE` function is that with a `HOP` you can "jump" forward in time. That's why you have to specify both the length of the window and the interval you want to jump forward. 13 | When using a `HOP` function, records can be assigned to multiple windows if the interval is smaller than the window length, like in this example. A tumbling window never overlaps and records will only belong to one window. 14 | 15 | ## Script 16 | 17 | ```sql 18 | CREATE TABLE bids ( 19 | bid_id STRING, 20 | currency_code STRING, 21 | bid_price DOUBLE, 22 | transaction_time TIMESTAMP(3), 23 | WATERMARK FOR transaction_time AS transaction_time - INTERVAL '5' SECONDS 24 | ) WITH ( 25 | 'connector' = 'faker', 26 | 'fields.bid_id.expression' = '#{Internet.UUID}', 27 | 'fields.currency_code.expression' = '#{regexify ''(EUR|USD|CNY)''}', 28 | 'fields.bid_price.expression' = '#{Number.randomDouble ''2'',''1'',''150''}', 29 | 'fields.transaction_time.expression' = '#{date.past ''30'',''SECONDS''}', 30 | 'rows-per-second' = '100' 31 | ); 32 | 33 | SELECT window_start, window_end, currency_code, ROUND(AVG(bid_price),2) AS MovingAverageBidPrice 34 | FROM TABLE( 35 | HOP(TABLE bids, DESCRIPTOR(transaction_time), INTERVAL '30' SECONDS, INTERVAL '1' MINUTE)) 36 | GROUP BY window_start, window_end, currency_code; 37 | ``` 38 | 39 | ## Example Output 40 | 41 | ![01_group_by_window](10_hopping_time_windows.png) 42 | -------------------------------------------------------------------------------- /joins/01_regular_joins/01_regular_joins.md: -------------------------------------------------------------------------------- 1 | # 01 Regular Joins 2 | 3 | > :bulb: This example will show how you can use joins to correlate rows across multiple tables. 4 | 5 | Flink SQL supports complex and flexible join operations over continuous tables. 6 | There are several different types of joins to account for the wide variety of semantics queries may require. 7 | 8 | Regular joins are the most generic and flexible type of join. 9 | These include the standard `INNER` and `[FULL|LEFT|RIGHT] OUTER` joins that are available in most modern databases. 10 | 11 | Suppose we have a [NOC list](https://en.wikipedia.org/wiki/Non-official_cover) of secret agents all over the world. 12 | Your mission if you choose to accept it, is to join this table with another containin the agents real name. 13 | 14 | In Flink SQL, this can be achieved using a simple `INNER JOIN`. 15 | Flink will join the tables using an equi-join predicate on the `agent_id` and output a new row everytime there is a match. 16 | 17 | However, there is something to be careful of. 18 | Flink must retain every input row as part of the join to potentially join it with the other table in the future. 19 | This means the queries resource requirements will grow indefinitely and will eventually fail. 20 | While this type of join is useful in some scenarios, other joins are more powerful in a streaming context and significantly more space-efficient. 21 | 22 | In this example, both tables are bounded to remain space efficient. 23 | 24 | ```sql 25 | CREATE TABLE NOC ( 26 | agent_id STRING, 27 | codename STRING 28 | ) 29 | WITH ( 30 | 'connector' = 'faker', 31 | 'fields.agent_id.expression' = '#{regexify ''(1|2|3|4|5){1}''}', 32 | 'fields.codename.expression' = '#{superhero.name}', 33 | 'number-of-rows' = '10' 34 | ); 35 | 36 | CREATE TABLE RealNames ( 37 | agent_id STRING, 38 | name STRING 39 | ) 40 | WITH ( 41 | 'connector' = 'faker', 42 | 'fields.agent_id.expression' = '#{regexify ''(1|2|3|4|5){1}''}', 43 | 'fields.name.expression' = '#{Name.full_name}', 44 | 'number-of-rows' = '10' 45 | ); 46 | 47 | SELECT 48 | name, 49 | codename 50 | FROM NOC 51 | INNER JOIN RealNames ON NOC.agent_id = RealNames.agent_id; 52 | ``` 53 | 54 | ![01_regular_joins](https://user-images.githubusercontent.com/23521087/105504538-280bab00-5cc8-11eb-962d-6f36432e422b.png) 55 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/conf/sql-client-conf.yaml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Copyright 2019 Ververica GmbH 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); 5 | # you may not use this file except in compliance with the License. 6 | # You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | ################################################################################ 16 | 17 | 18 | # This file defines the default environment for Flink's SQL Client. 19 | # Defaults might be overwritten by a session specific environment. 20 | 21 | #============================================================================== 22 | # Execution properties 23 | #============================================================================== 24 | 25 | # Execution properties allow for changing the behavior of a table program. 26 | 27 | execution: 28 | planner: blink # using the Blink planner 29 | type: streaming # 'batch' or 'streaming' execution 30 | result-mode: table # 'changelog' or 'table' presentation of results 31 | parallelism: 1 # parallelism of the program 32 | max-parallelism: 128 # maximum parallelism 33 | min-idle-state-retention: 0 # minimum idle state retention in ms 34 | max-idle-state-retention: 0 # maximum idle state retention in ms 35 | 36 | #============================================================================== 37 | # Deployment properties 38 | #============================================================================== 39 | 40 | # Deployment properties allow for describing the cluster to which table 41 | # programs are submitted to. 42 | 43 | deployment: 44 | type: standalone # only the 'standalone' deployment is supported 45 | response-timeout: 5000 # general cluster communication timeout in ms 46 | gateway-address: "" # (optional) address from cluster to gateway 47 | gateway-port: 0 # (optional) port from cluster to gateway 48 | 49 | 50 | -------------------------------------------------------------------------------- /foundations/07_views/07_views.md: -------------------------------------------------------------------------------- 1 | # 07 Encapsulating Logic with (Temporary) Views 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey) 4 | 5 | > :bulb: This example will show how you can use (temporary) views to reuse code and to structure long queries and scripts. 6 | 7 | `CREATE (TEMPORARY) VIEW` defines a view from a query. 8 | **A view is not physically materialized.** 9 | Instead, the query is run every time the view is referenced in a query. 10 | 11 | Temporary views are very useful to structure and decompose more complicated queries and to re-use queries within a longer script. 12 | Non-temporary views - stored in a persistent [catalog](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/catalogs.html) - can also be used to share common queries within your organization, e.g. common filters or pre-processing steps. 13 | 14 | Here, we create a view on the `server_logs` that only contains successful requests. 15 | This view encapsulates the logic of filtering the logs based on certain `status_code`s. 16 | This logic can subsequently be used by any query or script that has access to the catalog. 17 | 18 | ## Script 19 | 20 | ```sql 21 | CREATE TABLE server_logs ( 22 | client_ip STRING, 23 | client_identity STRING, 24 | userid STRING, 25 | user_agent STRING, 26 | log_time TIMESTAMP(3), 27 | request_line STRING, 28 | status_code STRING, 29 | size INT 30 | ) WITH ( 31 | 'connector' = 'faker', 32 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 33 | 'fields.client_identity.expression' = '-', 34 | 'fields.userid.expression' = '-', 35 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 36 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 37 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 38 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 39 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 40 | ); 41 | 42 | CREATE VIEW successful_requests AS 43 | SELECT * 44 | FROM server_logs 45 | WHERE status_code SIMILAR TO '[2,3][0-9][0-9]'; 46 | 47 | SELECT * FROM successful_requests; 48 | ``` 49 | 50 | ## Example Output 51 | 52 | ![views](https://user-images.githubusercontent.com/11538663/102009292-c5250c80-3d36-11eb-85b3-05b8faf8df5a.gif) 53 | 54 | -------------------------------------------------------------------------------- /aggregations-and-analytics/01_group_by_window/01_group_by_window.md: -------------------------------------------------------------------------------- 1 | # 01 Aggregating Time Series Data 2 | 3 | > :warning: This recipe is using a legacy function. We recommend following the [new recipe](01_group_by_window_tvf.md). 4 | 5 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | Many streaming applications work with time series data. 8 | To count the number of `DISTINCT` IP addresses seen each minute, rows need to be grouped based on a [time attribute](https://docs.ververica.com/user_guide/sql_development/table_view.html#time-attributes). 9 | Grouping based on time is special, because time always moves forward, which means Flink can generate final results after the minute is completed. 10 | 11 | `TUMBLE` is a built-in function for grouping timestamps into time intervals called windows. 12 | Unlike other aggregations, it will only produce a single final result for each key when the interval is completed. 13 | 14 | If the logs do not have a timestamp, one can be generated using a [computed column](https://docs.ververica.com/user_guide/sql_development/table_view.html#computed-column). 15 | `log_time AS PROCTIME()` will append a column to the table with the current system time. 16 | 17 | ## Script 18 | 19 | ```sql 20 | CREATE TABLE server_logs ( 21 | client_ip STRING, 22 | client_identity STRING, 23 | userid STRING, 24 | request_line STRING, 25 | status_code STRING, 26 | log_time AS PROCTIME() 27 | ) WITH ( 28 | 'connector' = 'faker', 29 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 30 | 'fields.client_identity.expression' = '-', 31 | 'fields.userid.expression' = '-', 32 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 33 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 34 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}' 35 | ); 36 | 37 | SELECT 38 | COUNT(DISTINCT client_ip) AS ip_addresses, 39 | TUMBLE_PROCTIME(log_time, INTERVAL '1' MINUTE) AS window_interval 40 | FROM server_logs 41 | GROUP BY 42 | TUMBLE(log_time, INTERVAL '1' MINUTE); 43 | ``` 44 | 45 | ## Example Output 46 | 47 | ![01_group_by_window](https://user-images.githubusercontent.com/23521087/105503522-fe05b900-5cc6-11eb-9c36-bd8dc2a8e9ce.png) 48 | -------------------------------------------------------------------------------- /aggregations-and-analytics/05_top_n/05_top_n.md: -------------------------------------------------------------------------------- 1 | # 05 Continuous Top-N 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.9%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to continuously calculate the "Top-N" rows based on a given attribute, using an `OVER` window and the `ROW_NUMBER()` function. 6 | 7 | The source table (`spells_cast`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | The Ministry of Magic tracks every spell a wizard casts throughout Great Britain and wants to know every wizard's Top 2 all-time favorite spells. 10 | 11 | Flink SQL can be used to calculate continuous [aggregations](../../foundations/05_group_by/05_group_by.md), so if we know 12 | each spell a wizard has cast, we can maintain a continuous total of how many times they have cast that spell. 13 | 14 | ```sql 15 | SELECT wizard, spell, COUNT(*) AS times_cast 16 | FROM spells_cast 17 | GROUP BY wizard, spell; 18 | ``` 19 | 20 | This result can be used in an `OVER` window to calculate a [Top-N](https://docs.ververica.com/user_guide/sql_development/queries.html#top-n). 21 | The rows are partitioned using the `wizard` column, and are then ordered based on the count of spell casts (`times_cast DESC`). 22 | The built-in function `ROW_NUMBER()` assigns a unique, sequential number to each row, starting from one, according to the rows' ordering within the partition. 23 | Finally, the results are filtered for only those rows with a `row_num <= 2` to find each wizard's top 2 favorite spells. 24 | 25 | Where Flink is most potent in this query is its ability to issue retractions. 26 | As wizards cast more spells, their top 2 will change. 27 | When this occurs, Flink will issue a retraction, modifying its output, so the result is always correct and up to date. 28 | 29 | 30 | ```sql 31 | CREATE TABLE spells_cast ( 32 | wizard STRING, 33 | spell STRING 34 | ) WITH ( 35 | 'connector' = 'faker', 36 | 'fields.wizard.expression' = '#{harry_potter.characters}', 37 | 'fields.spell.expression' = '#{harry_potter.spells}' 38 | ); 39 | 40 | SELECT wizard, spell, times_cast 41 | FROM ( 42 | SELECT *, 43 | ROW_NUMBER() OVER (PARTITION BY wizard ORDER BY times_cast DESC) AS row_num 44 | FROM (SELECT wizard, spell, COUNT(*) AS times_cast FROM spells_cast GROUP BY wizard, spell) 45 | ) 46 | WHERE row_num <= 2; 47 | ``` 48 | 49 | ![05_top_n](https://user-images.githubusercontent.com/23521087/105503736-3e653700-5cc7-11eb-9ddf-9a89d93841bc.png) 50 | -------------------------------------------------------------------------------- /aggregations-and-analytics/01_group_by_window/01_group_by_window_tvf.md: -------------------------------------------------------------------------------- 1 | # 01 Aggregating Time Series Data 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.13%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to aggregate time series data in real-time using a `TUMBLE` window. 6 | 7 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | Many streaming applications work with time series data. 10 | To count the number of `DISTINCT` IP addresses seen each minute, rows need to be grouped based on a [time attribute](https://docs.ververica.com/user_guide/sql_development/table_view.html#time-attributes). 11 | Grouping based on time is special, because time always moves forward, which means Flink can generate final results after the minute is completed. 12 | 13 | `TUMBLE` is a [built-in function](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/sql/queries/window-agg/) for grouping timestamps into time intervals called windows. Windows split the stream into “buckets” of finite size, over which we can apply computations. 14 | Unlike other aggregations such as `HOP` or `CUMULATE`, it will only produce a single final result for each key when the interval is completed. 15 | 16 | If the logs do not have a timestamp, one can be generated using a [computed column](https://docs.ververica.com/user_guide/sql_development/table_view.html#computed-column). 17 | `log_time AS PROCTIME()` will append a column to the table with the current system time. 18 | 19 | ## Script 20 | 21 | ```sql 22 | CREATE TABLE server_logs ( 23 | client_ip STRING, 24 | client_identity STRING, 25 | userid STRING, 26 | request_line STRING, 27 | status_code STRING, 28 | log_time AS PROCTIME() 29 | ) WITH ( 30 | 'connector' = 'faker', 31 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 32 | 'fields.client_identity.expression' = '-', 33 | 'fields.userid.expression' = '-', 34 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 35 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 36 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}' 37 | ); 38 | 39 | SELECT window_start, window_end, COUNT(DISTINCT client_ip) AS ip_addresses 40 | FROM TABLE( 41 | TUMBLE(TABLE server_logs, DESCRIPTOR(log_time), INTERVAL '1' MINUTE)) 42 | GROUP BY window_start, window_end; 43 | ``` 44 | 45 | ## Example Output 46 | 47 | ![01_group_by_window](01_group_by_window_tvf_result.png) 48 | -------------------------------------------------------------------------------- /aggregations-and-analytics/04_over/04_over.md: -------------------------------------------------------------------------------- 1 | # 04 Rolling Aggregations on Time Series Data 2 | 3 | > :bulb: This example will show how to calculate an aggregate or cumulative value based on a group of rows using an `OVER` window. A typical use case are rolling aggregations. 4 | 5 | The source table (`temperature_measurements`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | OVER window aggregates compute an aggregated value for every input row over a range of ordered rows. 8 | In contrast to GROUP BY aggregates, OVER aggregates do not reduce the number of result rows to a single row for every group. 9 | Instead, OVER aggregates produce an aggregated value for every input row. 10 | 11 | The order needs to be defined by a [time attribute](https://docs.ververica.com/user_guide/sql_development/table_view.html#time-attributes). 12 | The range of rows can be defined by a number of rows or a time interval. 13 | 14 | In this example, we are trying to identify outliers in the `temperature_measurements` table. 15 | For this, we use an `OVER` window to calculate, for each measurement, the maximum (`MAX`), minimum (`MIN`) and average (`AVG`) temperature across all measurements, as well as the standard deviation (`STDDEV`), for the same city over the previous minute. 16 | > As an exercise, you can try to write another query to filter out any temperature measurement that are higher or lower than the average by more than four standard deviations. 17 | 18 | ## Script 19 | 20 | ```sql 21 | CREATE TEMPORARY TABLE temperature_measurements ( 22 | measurement_time TIMESTAMP(3), 23 | city STRING, 24 | temperature FLOAT, 25 | WATERMARK FOR measurement_time AS measurement_time - INTERVAL '15' SECONDS 26 | ) 27 | WITH ( 28 | 'connector' = 'faker', 29 | 'fields.measurement_time.expression' = '#{date.past ''15'',''SECONDS''}', 30 | 'fields.temperature.expression' = '#{number.numberBetween ''0'',''50''}', 31 | 'fields.city.expression' = '#{regexify ''(Chicago|Munich|Berlin|Portland|Hangzhou|Seatle|Beijing|New York){1}''}' 32 | ); 33 | 34 | SELECT 35 | measurement_time, 36 | city, 37 | temperature, 38 | AVG(CAST(temperature AS FLOAT)) OVER last_minute AS avg_temperature_minute, 39 | MAX(temperature) OVER last_minute AS min_temperature_minute, 40 | MIN(temperature) OVER last_minute AS max_temperature_minute, 41 | STDDEV(CAST(temperature AS FLOAT)) OVER last_minute AS stdev_temperature_minute 42 | FROM temperature_measurements 43 | WINDOW last_minute AS ( 44 | PARTITION BY city 45 | ORDER BY measurement_time 46 | RANGE BETWEEN INTERVAL '1' MINUTE PRECEDING AND CURRENT ROW 47 | ); 48 | ``` 49 | ## Example Output 50 | 51 | ![04_over](https://user-images.githubusercontent.com/23521087/105503670-2beafd80-5cc7-11eb-9e58-7a4ed71b1d7c.png) 52 | -------------------------------------------------------------------------------- /joins/02_interval_joins/02_interval_joins.md: -------------------------------------------------------------------------------- 1 | # 02 Interval Joins 2 | 3 | > :bulb: This example will show how you can perform joins between tables with events that are related in a temporal context. 4 | 5 | ## Why Interval Joins? 6 | 7 | In a [previous recipe](../01_regular_joins/01_regular_joins.md), you learned about using _regular joins_ in Flink SQL. This kind of join works well for some scenarios, but for others a more efficient type of join is required to keep resource utilization from growing indefinitely. 8 | 9 | One of the ways to optimize joining operations in Flink SQL is to use [_interval joins_](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/streaming/joins.html#interval-joins). An interval join is defined by a join predicate that checks if the time attributes of the input events are within certain time constraints (i.e. a time window). 10 | 11 | ## Using Interval Joins 12 | 13 | Suppose you want to join events of two tables that correlate to each other in the [order fulfillment lifecycle](https://en.wikipedia.org/wiki/Order_fulfillment) (`orders` and `shipments`) and that are under a Service-level Aggreement (SLA) of **3 days**. To reduce the amount of input rows Flink has to retain and optimize the join operation, you can define a time constraint in the `WHERE` clause to bound the time on both sides to that specific interval using a `BETWEEN` predicate. 14 | 15 | ## Script 16 | 17 | The source tables (`orders` and `shipments`) are backed by the built-in [`datagen` connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/datagen.html), which continuously generates rows in memory. 18 | 19 | ```sql 20 | CREATE TABLE orders ( 21 | id INT, 22 | order_time AS TIMESTAMPADD(DAY, CAST(FLOOR(RAND()*(1-5+1)+5)*(-1) AS INT), CURRENT_TIMESTAMP) 23 | ) 24 | WITH ( 25 | 'connector' = 'datagen', 26 | 'rows-per-second'='10', 27 | 'fields.id.kind'='sequence', 28 | 'fields.id.start'='1', 29 | 'fields.id.end'='1000' 30 | ); 31 | 32 | 33 | CREATE TABLE shipments ( 34 | id INT, 35 | order_id INT, 36 | shipment_time AS TIMESTAMPADD(DAY, CAST(FLOOR(RAND()*(1-5+1)) AS INT), CURRENT_TIMESTAMP) 37 | ) 38 | WITH ( 39 | 'connector' = 'datagen', 40 | 'rows-per-second'='5', 41 | 'fields.id.kind'='random', 42 | 'fields.id.min'='0', 43 | 'fields.order_id.kind'='sequence', 44 | 'fields.order_id.start'='1', 45 | 'fields.order_id.end'='1000' 46 | ); 47 | 48 | SELECT 49 | o.id AS order_id, 50 | o.order_time, 51 | s.shipment_time, 52 | TIMESTAMPDIFF(DAY,o.order_time,s.shipment_time) AS day_diff 53 | FROM orders o 54 | JOIN shipments s ON o.id = s.order_id 55 | WHERE 56 | o.order_time BETWEEN s.shipment_time - INTERVAL '3' DAY AND s.shipment_time; 57 | ``` 58 | 59 | ## Example Output 60 | 61 | ![15_interval_joins](https://user-images.githubusercontent.com/23521087/102237138-9ce30c80-3ef4-11eb-969f-8f157b249ebb.png) 62 | -------------------------------------------------------------------------------- /foundations/05_group_by/05_group_by.md: -------------------------------------------------------------------------------- 1 | # 05 Aggregating Data 2 | 3 | > :bulb: This example will show how to aggregate server logs in real-time using the standard `GROUP BY` clause. 4 | 5 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | To count the number of logs received per browser for each status code _over time_, you can combine the `COUNT` aggregate function with a `GROUP BY` clause. Because the `user_agent` field contains a lot of information, you can extract the browser using the built-in [string function](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/functions/systemfunctions/#string-functions) `REGEXP_EXTRACT`. 8 | 9 | A `GROUP BY` on a streaming table produces an updating result, so you will see the aggregated count for each browser continuously changing as new rows flow in. 10 | 11 | > As an exercise, you can play around with other standard SQL aggregate functions (e.g. `SUM`,`AVG`,`MIN`,`MAX`). 12 | 13 | ## Script 14 | 15 | ```sql 16 | CREATE TABLE server_logs ( 17 | client_ip STRING, 18 | client_identity STRING, 19 | userid STRING, 20 | user_agent STRING, 21 | log_time TIMESTAMP(3), 22 | request_line STRING, 23 | status_code STRING, 24 | size INT 25 | ) WITH ( 26 | 'connector' = 'faker', 27 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 28 | 'fields.client_identity.expression' = '-', 29 | 'fields.userid.expression' = '-', 30 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 31 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 32 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 33 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 34 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 35 | ); 36 | 37 | -- Sample user_agent: Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_3) AppleWebKit/537.75.14 (KHTML, like Gecko) Version/7.0.3 Safari/7046A194A 38 | -- Regex pattern: '[^\/]+' (Match everything before '/') 39 | SELECT 40 | REGEXP_EXTRACT(user_agent,'[^\/]+') AS browser, 41 | status_code, 42 | COUNT(*) AS cnt_status 43 | FROM server_logs 44 | GROUP BY 45 | REGEXP_EXTRACT(user_agent,'[^\/]+'), 46 | status_code; 47 | ``` 48 | 49 | ## Example Output 50 | 51 | This example can be run in the [SQL Client](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sqlClient.html), a command line tool to develop and execute Flink SQL queries that is bundled in Flink. 52 | 53 | ![03_group_by](https://user-images.githubusercontent.com/23521087/101014385-19293780-3566-11eb-9d81-9c99d6ffa7e4.gif) 54 | -------------------------------------------------------------------------------- /other-builtin-functions/04_override_table_options/04_override_table_options.md: -------------------------------------------------------------------------------- 1 | # 04 Overriding table options 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey) 4 | 5 | > :bulb: This example will show how you can override table options that have been defined via a DDL by using Hints. 6 | 7 | This recipe uses the `2015 Flight Delays and Cancellations` dataset which can be found on [Kaggle](https://www.kaggle.com/usdot/flight-delays). 8 | 9 | As explained before in the [creating tables recipe](../../foundations/01_create_table/01_create_table.md), you create tables in Flink SQL by using a SQL DDL. For example, you would use the following DDL to create a table `airports` which reads available airports in via the provided CSV file. 10 | 11 | > :warning: Make sure that the value for `path` is correct for your location environment. 12 | 13 | ```sql 14 | CREATE TABLE `airports` ( 15 | `IATA_CODE` CHAR(3), 16 | `AIRPORT` STRING, 17 | `CITY` STRING, 18 | `STATE` CHAR(2), 19 | `COUNTRY` CHAR(3), 20 | `LATITUDE` DOUBLE NULL, 21 | `LONGITUDE` DOUBLE NULL, 22 | PRIMARY KEY (`IATA_CODE`) NOT ENFORCED 23 | ) WITH ( 24 | 'connector' = 'filesystem', 25 | 'path' = 'file:///flink-sql-cookbook/other-builtin-functions/04_override_table_options/airports.csv', 26 | 'format' = 'csv' 27 | ); 28 | ``` 29 | 30 | After creating this table, you would normally query it using something like: 31 | 32 | ```sql 33 | SELECT * FROM `airports`; 34 | ``` 35 | 36 | However, this currently doesn't work because there is an improperly formatted line in the CSV file. There is an option for CSV files to ignore parsing errors, but that means you need to alter the table. 37 | 38 | You can also override the defined table options using [SQL Hints](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/sql/queries/hints/). Your SQL statement would then look like: 39 | 40 | ```sql 41 | SELECT * FROM `airports` /*+ OPTIONS('csv.ignore-parse-errors'='true') */; 42 | ``` 43 | 44 | Since the CSV format option `csv.ignore-parse-errors` sets fields to null in case of errors, you can also quickly identify which fields can't be parsed using: 45 | 46 | ```sql 47 | SELECT * FROM `airports` /*+ OPTIONS('csv.ignore-parse-errors'='true') */ WHERE `LATITUDE` IS NULL; 48 | ``` 49 | 50 | You can apply SQL Hints for all possible table options. For example, if you SQL job which reads from Kafka has crashed, you can override the default reading position: 51 | 52 | ```sql 53 | SELECT * FROM `your_kafka_topic` /*+ OPTIONS('scan.startup.mode'='group-offsets') */; 54 | ``` 55 | 56 | Tables, views and functions are all registered in the catalog. The catalog is a collection of metadata. Using SQL Hints, you can override any defined metadata. 57 | 58 | ## Example Output 59 | 60 | ![04_override_table_options.screen01](04_override_table_options.screen01.png) 61 | ![04_override_table_options.screen02](04_override_table_options.screen02.png) -------------------------------------------------------------------------------- /aggregations-and-analytics/02_watermarks/02_watermarks.md: -------------------------------------------------------------------------------- 1 | # 02 Watermarks 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.10%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to use `WATERMARK`s to work with timestamps in records. 6 | 7 | The source table (`doctor_sightings`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | The [previous recipe](../01_group_by_window/01_group_by_window.md) showed how a `TUMBLE` group window makes it simple to aggregate time-series data. 10 | 11 | [The Doctor](https://tardis.fandom.com/wiki/The_Doctor) is a renegade time lord who travels through space and time in a [TARDIS](https://tardis.fandom.com/wiki/The_Doctor%27s_TARDIS). 12 | As different versions of the Doctor travel through time, various people log their sightings. 13 | We want to track how many times each version of the Doctor is seen each minute. 14 | Unlike the previous recipe, these records have an embedded timestamp we need to use to perform our calculation. 15 | 16 | More often than not, most data will come with embedded timestamps that we want to use for our time series calculations. We call this timestamp an [event-time attribute](https://ci.apache.org/projects/flink/flink-docs-stable/docs/learn-flink/streaming_analytics/#event-time-and-watermarks). 17 | 18 | Event time represents when something actually happened in the real world. 19 | And it is unique because it is quasi-monotonically increasing; we generally see things that happened earlier before seeing things that happen later. Of course, data will never be perfectly ordered (systems go down, networks are laggy, doctor sighting take time to postmark and mail), and there will be some out-of-orderness in our data. 20 | 21 | Flink can account for all these variabilities using a [WATERMARK](https://docs.ververica.com/user_guide/sql_development/table_view.html#event-time) attribute in the tables DDL. The watermark signifies a column as the table's event time attribute and tells Flink how out of order we expect our data. 22 | 23 | In the Doctor's case, we expect all records to arrive within 15 seconds when the sighting occurs. 24 | 25 | ## Script 26 | 27 | ```sql 28 | CREATE TABLE doctor_sightings ( 29 | doctor STRING, 30 | sighting_time TIMESTAMP(3), 31 | WATERMARK FOR sighting_time AS sighting_time - INTERVAL '15' SECONDS 32 | ) 33 | WITH ( 34 | 'connector' = 'faker', 35 | 'fields.doctor.expression' = '#{dr_who.the_doctors}', 36 | 'fields.sighting_time.expression' = '#{date.past ''15'',''SECONDS''}' 37 | ); 38 | 39 | SELECT 40 | doctor, 41 | TUMBLE_ROWTIME(sighting_time, INTERVAL '1' MINUTE) AS sighting_time, 42 | COUNT(*) AS sightings 43 | FROM doctor_sightings 44 | GROUP BY 45 | TUMBLE(sighting_time, INTERVAL '1' MINUTE), 46 | doctor; 47 | ``` 48 | 49 | ## Example Output 50 | 51 | ![02_watermarks](https://user-images.githubusercontent.com/23521087/105503592-12e24c80-5cc7-11eb-9155-243cc9c314f0.png) 52 | -------------------------------------------------------------------------------- /foundations/03_temporary_table/03_temporary_table.md: -------------------------------------------------------------------------------- 1 | # 03 Working with Temporary Tables 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey) 4 | 5 | > :bulb: This example will show how and why to create a temporary table using SQL DDL. 6 | 7 | Non-temporary tables in Flink SQL are stored in a catalog, while temporary tables only live within the current session (Apache Flink CLI) or script (Ververica Platform). 8 | You can use a temporary table instead of a regular (catalog) table, if it is only meant to be used within the current session or script. 9 | 10 | This example is exactly the same as [Inserting Into Tables](../02_insert_into/02_insert_into.md) except that both `server_logs` and `client_errors` are created as temporary tables. 11 | 12 | ### Why Temporary Tables? 13 | 14 | For result tables like `client_errors` that no one can ever read from (because of its type `blackhole`) it makes a lot of sense to use a temporary table instead of publishing its metadata in a catalog. 15 | 16 | Furthermore, temporary tables allow you to create fully self-contained scripts, which is why we will mostly use those in the Flink SQL Cookbook. 17 | 18 | ## Script 19 | 20 | ```sql 21 | 22 | CREATE TEMPORARY TABLE server_logs ( 23 | client_ip STRING, 24 | client_identity STRING, 25 | userid STRING, 26 | user_agent STRING, 27 | log_time TIMESTAMP(3), 28 | request_line STRING, 29 | status_code STRING, 30 | size INT 31 | ) WITH ( 32 | 'connector' = 'faker', 33 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 34 | 'fields.client_identity.expression' = '-', 35 | 'fields.userid.expression' = '-', 36 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 37 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 38 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 39 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 40 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 41 | ); 42 | 43 | CREATE TEMPORARY TABLE client_errors ( 44 | log_time TIMESTAMP(3), 45 | request_line STRING, 46 | status_code STRING, 47 | size INT 48 | ) 49 | WITH ( 50 | 'connector' = 'blackhole' 51 | ); 52 | 53 | INSERT INTO client_errors 54 | SELECT 55 | log_time, 56 | request_line, 57 | status_code, 58 | size 59 | FROM server_logs 60 | WHERE 61 | status_code SIMILAR TO '4[0-9][0-9]'; 62 | ``` 63 | 64 | ## Example Output 65 | 66 | In comparison to [Inserting Into Tables](../02_insert_into/02_insert_into.md), you can see that the two temporary tables do not appear in the catalog browser on the left. 67 | The table definitions never make it into the catalog, but are just submitted as part of the script that contains the `INSERT INTO` statement. 68 | 69 | ![Screencast GIF](https://user-images.githubusercontent.com/11538663/101192652-aac6a100-365b-11eb-82a3-5b86522e772c.gif) 70 | -------------------------------------------------------------------------------- /joins/06_lateral_join/06_lateral_join.md: -------------------------------------------------------------------------------- 1 | # 06 Lateral Table Join 2 | 3 | > :bulb: This example will show how you can correlate events using a `LATERAL` join. 4 | 5 | A recent addition to the SQL standard is the `LATERAL` join, which allows you to combine 6 | the power of a correlated subquery with the expressiveness of a join. 7 | 8 | Given a table with people's addresses, you need to find the two most populous cities 9 | for each state and continuously update those rankings as people move. The input table 10 | of `People` contains a uid for each person and their address and when they moved there. 11 | 12 | The first step is to calculate each city's population using a [continuous aggregation](../../foundations/05_group_by/05_group_by.md). 13 | While this is simple enough, the real power of Flink SQL comes when people move. By using 14 | [deduplication](../../aggregations-and-analytics/06_dedup/06_dedup.md) Flink will automatically issue a retraction for a persons old city when 15 | they move. So if John moves from New York to Los Angelos, the population for New York will 16 | automatically go down by 1. This gives us the power change-data-capture without having 17 | to invest in the actual infrastructure of setting it up! 18 | 19 | With this dynamic population table at hand, you are ready to solve the original problem using a `LATERAL` table join. 20 | Unlike a normal join, lateral joins allow the subquery to correlate with columns from other arguments in the `FROM` clause. And unlike a regular subquery, as a join, the lateral can return multiple rows. 21 | You can now have a sub-query correlated with every individual state, and for every state it ranks by population and returns the top 2 cities. 22 | 23 | ## Script 24 | 25 | ```sql 26 | CREATE TABLE People ( 27 | id INT, 28 | city STRING, 29 | state STRING, 30 | arrival_time TIMESTAMP(3), 31 | WATERMARK FOR arrival_time AS arrival_time - INTERVAL '1' MINUTE 32 | ) WITH ( 33 | 'connector' = 'faker', 34 | 'fields.id.expression' = '#{number.numberBetween ''1'',''100''}', 35 | 'fields.city.expression' = '#{regexify ''(Newmouth|Newburgh|Portport|Southfort|Springfield){1}''}', 36 | 'fields.state.expression' = '#{regexify ''(New York|Illinois|California|Washington){1}''}', 37 | 'fields.arrival_time.expression' = '#{date.past ''15'',''SECONDS''}', 38 | 'rows-per-second' = '10' 39 | ); 40 | 41 | CREATE TEMPORARY VIEW CurrentPopulation AS 42 | SELECT 43 | city, 44 | state, 45 | COUNT(*) as population 46 | FROM ( 47 | SELECT 48 | city, 49 | state, 50 | ROW_NUMBER() OVER (PARTITION BY id ORDER BY arrival_time DESC) AS rownum 51 | FROM People 52 | ) 53 | WHERE rownum = 1 54 | GROUP BY city, state; 55 | 56 | SELECT 57 | state, 58 | city, 59 | population 60 | FROM 61 | (SELECT DISTINCT state FROM CurrentPopulation) States, 62 | LATERAL ( 63 | SELECT city, population 64 | FROM CurrentPopulation 65 | WHERE state = States.state 66 | ORDER BY population DESC 67 | LIMIT 2 68 | ); 69 | ``` 70 | 71 | ## Example Output 72 | 73 | ![lateral](https://user-images.githubusercontent.com/23521087/105504738-6bfeb000-5cc8-11eb-9517-1242dfa87bb4.gif) 74 | -------------------------------------------------------------------------------- /foundations/02_insert_into/02_insert_into.md: -------------------------------------------------------------------------------- 1 | # 02 Inserting Into Tables 2 | 3 | > :bulb: This recipe shows how to insert rows into a table so that downstream applications can read them. 4 | 5 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | As outlined in [the first recipe](../01_create_table/01_create_table.md) Flink SQL operates on tables, that are stored in external systems. 8 | To publish results of a query for consumption by downstream applications, you write the results of a query into a table. 9 | This table can be read by Flink SQL, or directly by connecting to the external system that is storing the data (e.g. an ElasticSearch index.) 10 | 11 | This example takes the `server_logs` tables, filters for client errors, and writes these logs into another table called `client_errors`. 12 | Any number of external systems could back the result table, including Apache Kafka, Apache Hive, ElasticSearch, JDBC, among many others. 13 | To keep this example self-contained, `client_errors` is of type `blackhole`: instead of actually writing the data to an external system, the table discards any rows written to it. 14 | 15 | ## Script 16 | 17 | ```sql 18 | CREATE TABLE server_logs ( 19 | client_ip STRING, 20 | client_identity STRING, 21 | userid STRING, 22 | user_agent STRING, 23 | log_time TIMESTAMP(3), 24 | request_line STRING, 25 | status_code STRING, 26 | size INT 27 | ) WITH ( 28 | 'connector' = 'faker', 29 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 30 | 'fields.client_identity.expression' = '-', 31 | 'fields.userid.expression' = '-', 32 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 33 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 34 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 35 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 36 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 37 | ); 38 | 39 | CREATE TABLE client_errors ( 40 | log_time TIMESTAMP(3), 41 | request_line STRING, 42 | status_code STRING, 43 | size INT 44 | ) 45 | WITH ( 46 | 'connector' = 'blackhole' 47 | ); 48 | 49 | INSERT INTO client_errors 50 | SELECT 51 | log_time, 52 | request_line, 53 | status_code, 54 | size 55 | FROM server_logs 56 | WHERE 57 | status_code SIMILAR TO '4[0-9][0-9]'; 58 | ``` 59 | 60 | ## Example Output 61 | 62 | An INSERT INTO query that reads from an unbounded table (like `server_logs`) is a long-running application. 63 | When you run such a statement in Apache Flink's [SQL Client](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sqlClient.html) a Flink Job will be submitted to the configured cluster. 64 | In Ververica Platform a so called Deployment will be created to manage the execution of the statement. 65 | 66 | ![Screenshot GIF](https://user-images.githubusercontent.com/11538663/101192280-22480080-365b-11eb-97e9-35f151027c6e.gif) 67 | -------------------------------------------------------------------------------- /other-builtin-functions/05_expanding_arrays/05_expanding_arrays.md: -------------------------------------------------------------------------------- 1 | # 05 Expanding arrays into new rows 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.3%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to create new rows for each element in an array using a `CROSS JOIN UNNEST`. 6 | 7 | The source table (`HarryPotter`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | There are many cases where data contains complex data types. Complex data types means that the data is structured in a nested way. Examples of these data types are `ARRAY`, `MAP` or `ROW`. 10 | 11 | In this recipe, we'll expand an `ARRAY` into a new row for each element using `CROSS JOIN UNNEST` and join the new rows with a lookup table. 12 | 13 | This table DDL creates a `HarryPotter` themed table. It contains a character from Harry Potter and 3 spells that the character used. The `spells` is of data type `ARRAY`. 14 | 15 | ## Script 16 | 17 | ```sql 18 | -- Create source table 19 | CREATE TABLE `HarryPotter` ( 20 | `character` STRING, 21 | `spells` ARRAY 22 | ) WITH ( 23 | 'connector' = 'faker', 24 | 'fields.character.expression' = '#{harry_potter.character}', 25 | 'fields.spells.expression' = '#{harry_potter.spell}', 26 | 'fields.spells.length' = '3' 27 | ); 28 | ``` 29 | 30 | When querying this table, your results will look like this: 31 | 32 | ![05_complex_data_types](05_complex_data_types.png) 33 | 34 | In order to generate new rows for each element in the `spells` array, we’ll use a `CROSS JOIN UNNEST`. By applying this statement, the `UNNEST` will create one row for each element in `spells`, which we will store in a temporary table `SpellsTable`. Secondly, the `CROSS JOIN` joins each row in the `SpellsTable` with the matching row of the `HarryPotter` table. The DDL below will create a view that we’ll use to join all the newly generated rows with a lookup table. 35 | 36 | ```sql 37 | CREATE TEMPORARY VIEW `SpellsPerCharacter` AS 38 | SELECT `HarryPotter`.`character`, `SpellsTable`.`spell` 39 | FROM HarryPotter 40 | CROSS JOIN UNNEST(HarryPotter.spells) AS SpellsTable (spell); 41 | ``` 42 | 43 | The results of this view will contain each individual spell as a row. The results look like the following: 44 | 45 | ![05_unnested_data](05_unnested_data.png) 46 | 47 | You can then use the resulting table like you normally would, for example, for joining with a lookup table. Here, we enrich each `spell` with the `spoken_language` so that you can see in which language the spells were cast. 48 | 49 | ```sql 50 | CREATE TABLE `Spells_Language` ( 51 | `spells` STRING, 52 | `spoken_language` STRING, 53 | `proctime` AS PROCTIME() 54 | ) 55 | WITH ( 56 | 'connector' = 'faker', 57 | 'fields.spells.expression' = '#{harry_potter.spell}', 58 | 'fields.spoken_language.expression' = '#{regexify ''(Parseltongue|Rune|Gobbledegook|Mermish|Troll|English)''}' 59 | ); 60 | ``` 61 | 62 | ```sql 63 | SELECT 64 | `SpellsPerCharacter`.`character`, 65 | `SpellsPerCharacter`.`spell`, 66 | `Spells_Language`.`spoken_language` 67 | FROM SpellsPerCharacter 68 | JOIN Spells_Language FOR SYSTEM_TIME AS OF proctime AS Spells_Language 69 | ON SpellsPerCharacter.spell = Spells_Language.spells; 70 | ``` 71 | 72 | ## Example Output 73 | 74 | ![05_joined_unnested_data](05_joined_unnested_data.png) -------------------------------------------------------------------------------- /other-builtin-functions/06_split_strings_into_maps/06_split_strings_into_maps.md: -------------------------------------------------------------------------------- 1 | # 06 Split strings into maps 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.3%2B-lightgrey) 4 | 5 | > :bulb: This example will show how you can create a map of key/value pairs by splitting string values using `STR_TO_MAP`. 6 | 7 | The source table (`customers`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | There are many different data types in Flink SQL. You can group these in Character Strings, Binary Strings, Exact Numerics, Approximate Numerics, Date and Time, Constructed Data Types, User-Defined Types and Other Data Types. 10 | Some examples are `VARCHAR/STRING`, `CHAR`, `DECIMAL`, `DATE`, `TIME`, `TIMESTAMP`, `ARRAY`, `MAP`, `ROW` and `JSON`. You can find more information about these data types in the [Flink SQL Data Types Reference](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/types/). 11 | 12 | In this recipe, we'll convert two `STRING` columns containing a `postal_address` and a `residential_address` into a `MAP` column. 13 | 14 | This table DDL creates a `customers` table. It contains an identifier, the full name of a customer, the address to which you sent mail and the address where the customer is living. 15 | 16 | ## Script 17 | 18 | ```sql 19 | -- Create source table 20 | CREATE TABLE `customers` ( 21 | `identifier` STRING, 22 | `fullname` STRING, 23 | `postal_address` STRING, 24 | `residential_address` STRING 25 | ) WITH ( 26 | 'connector' = 'faker', 27 | 'fields.identifier.expression' = '#{Internet.uuid}', 28 | 'fields.fullname.expression' = '#{Name.firstName} #{Name.lastName}', 29 | 'fields.postal_address.expression' = '#{Address.fullAddress}', 30 | 'fields.residential_address.expression' = '#{Address.fullAddress}', 31 | 'rows-per-second' = '1' 32 | ); 33 | ``` 34 | 35 | After creating this table, we use the [`STR_TO_MAP`](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/functions/systemfunctions/#string-functions) in our SELECT statement. 36 | This function splits a `STRING` value into one or more key/value pair(s) using a delimiter. 37 | The default pair delimiter is `,` but this can be adjusted by providing a second argument to this function. In this example, we change the pair delimiter to `;` since our addresses can contain `,`. 38 | There is also a default key-value delimiter, which is `=`. In this example, we're changing this to `:` by providing a third argument to the function. 39 | 40 | To create our `MAP` column, we're using `||` to concatenate multiple `STRING` values. 41 | We're hardcoding the first key to 'postal_address:' to include the key-value delimiter and concatenate the value from the `postal_address` column. 42 | We then continue with hardcoding our second key to ';residential_address:'. That includes the pair delimiter `;` as a prefix and again `:` as our key-value delimiter as a suffix. 43 | To complete the function, we change the default values for pair delimiter and key-value delimiter to `;` and `:` respectively. 44 | 45 | ```sql 46 | SELECT 47 | `identifier`, 48 | `fullname`, 49 | STR_TO_MAP('postal_address:' || postal_address || ';residential_address:' || residential_address,';',':') AS `addresses` 50 | FROM `customers`; 51 | ``` 52 | 53 | ## Example Output 54 | 55 | ![06_create_maps](06_split_strings_into_maps.png) -------------------------------------------------------------------------------- /other-builtin-functions/02_union-all/02_union-all.md: -------------------------------------------------------------------------------- 1 | # 02 Building the Union of Multiple Streams 2 | 3 | > :bulb: This example will show how you can use the set operation `UNION ALL` to combine several streams of data. 4 | 5 | See [our documentation](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/sql/queries/set-ops/) 6 | for a full list of fantastic set operations Apache Flink supports. 7 | 8 | 9 | ## The Sources 10 | 11 | The examples assumes you are building an application that is tracking visits :fox_face: on foreign planets :chestnut:. 12 | There are three sources of visits. The universe of Rick and Morty, the very real world of NASA and such, 13 | and the not so real world of Hitchhikers Guide To The Galaxy. 14 | 15 | All three tables are `unbounded` and backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker). 16 | 17 | All sources of tracked visits have the `location` and `visit_time` in common. Some have `visitors`, some have 18 | `spacecrafts` and one has both. 19 | 20 | ```sql 21 | CREATE TEMPORARY TABLE rickandmorty_visits ( 22 | visitor STRING, 23 | location STRING, 24 | visit_time TIMESTAMP(3) 25 | ) WITH ( 26 | 'connector' = 'faker', 27 | 'fields.visitor.expression' = '#{RickAndMorty.character}', 28 | 'fields.location.expression' = '#{RickAndMorty.location}', 29 | 'fields.visit_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}' 30 | ); 31 | 32 | CREATE TEMPORARY TABLE spaceagency_visits ( 33 | spacecraft STRING, 34 | location STRING, 35 | visit_time TIMESTAMP(3) 36 | ) WITH ( 37 | 'connector' = 'faker', 38 | 'fields.spacecraft.expression' = '#{Space.nasaSpaceCraft}', 39 | 'fields.location.expression' = '#{Space.star}', 40 | 'fields.visit_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}' 41 | ); 42 | 43 | CREATE TEMPORARY TABLE hitchhiker_visits ( 44 | visitor STRING, 45 | starship STRING, 46 | location STRING, 47 | visit_time TIMESTAMP(3) 48 | ) WITH ( 49 | 'connector' = 'faker', 50 | 'fields.visitor.expression' = '#{HitchhikersGuideToTheGalaxy.character}', 51 | 'fields.starship.expression' = '#{HitchhikersGuideToTheGalaxy.starship}', 52 | 'fields.location.expression' = '#{HitchhikersGuideToTheGalaxy.location}', 53 | 'fields.visit_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}' 54 | ); 55 | 56 | ``` 57 | 58 | ## The Query 59 | 60 | We are using `UNION ALL` because it doesn't try to combine equivalent rows like 61 | `UNION` would do. That is also the reason why `UNION` can only be used with `bounded` streams. 62 | 63 | 64 | ```sql 65 | SELECT visitor, '' AS spacecraft, location, visit_time FROM rickandmorty_visits 66 | UNION ALL 67 | SELECT '' AS visitor, spacecraft, location, visit_time FROM spaceagency_visits 68 | UNION ALL 69 | SELECT visitor, starship AS spacecraft, location, visit_time FROM hitchhiker_visits; 70 | ``` 71 | 72 | :alien: As we are using `CREATE TEMPORARY TABLE`, you need to run both the `CREATE TABLE` and the `SELECT` statements together. 73 | 74 | ## The Beauty in VVP 75 | 76 | ![screeny](https://user-images.githubusercontent.com/68620/108173907-081cab00-70ff-11eb-823a-8a245b390485.png) 77 | 78 | 79 | The result is a combined stream of people visiting a location in one of those fantastic universes. 80 | We are sure you'll understand why this is one of our favorite queries. 81 | 82 | :bird: [Let us know](https://twitter.com/ververicadata) about your favorite streaming SQL Query. 83 | -------------------------------------------------------------------------------- /joins/04_lookup_joins/04_lookup_joins.md: -------------------------------------------------------------------------------- 1 | # 04 Lookup Joins 2 | 3 | > :bulb: This example will show how you can enrich a stream with an external table of reference data (i.e. a _lookup_ table). 4 | 5 | ## Data Enrichment 6 | 7 | Not all data changes frequently, even when working in real-time: in some cases, you might need to enrich streaming data with static — or _reference_ — data that is stored externally. 8 | For example, `user` metadata may be stored in a relational database that Flink needs to join against directly. 9 | Flink SQL allows you to look up reference data and join it with a stream using a _lookup join_. The join requires one table to have a [processing time attribute](https://docs.ververica.com/user_guide/sql_development/table_view.html#processing-time-attributes) and the other table to be backed by a [lookup source connector](https://docs.ververica.com/user_guide/sql_development/connectors.html#id1), like the JDBC connector. 10 | 11 | ## Using Lookup Joins 12 | 13 | In this example, you will look up reference user data stored in MySQL to flag subscription events for users that are minors (`age < 18`). The `FOR SYSTEM_TIME AS OF` clause uses the processing time attribute to ensure that each row of the `subscriptions` table is joined with the `users` rows that match the join predicate at the point in time when the `subscriptions` row is processed by the join operator. The lookup join also requires an equality join predicate based on the `PRIMARY KEY` of the lookup table (`usub.user_id = u.user_id`). Here, the source does not have to read the entire table and can lazily fetch individual values from the external table when necessary. 14 | 15 | ## Script 16 | 17 | The source table (`subscriptions`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. The `users` table is backed by an existing MySQL reference table using the [JDBC connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/jdbc.html). 18 | 19 | ```sql 20 | CREATE TABLE subscriptions ( 21 | id STRING, 22 | user_id INT, 23 | type STRING, 24 | start_date TIMESTAMP(3), 25 | end_date TIMESTAMP(3), 26 | payment_expiration TIMESTAMP(3), 27 | proc_time AS PROCTIME() 28 | ) WITH ( 29 | 'connector' = 'faker', 30 | 'fields.id.expression' = '#{Internet.uuid}', 31 | 'fields.user_id.expression' = '#{number.numberBetween ''1'',''50''}', 32 | 'fields.type.expression'= '#{regexify ''(basic|premium|platinum){1}''}', 33 | 'fields.start_date.expression' = '#{date.past ''30'',''DAYS''}', 34 | 'fields.end_date.expression' = '#{date.future ''365'',''DAYS''}', 35 | 'fields.payment_expiration.expression' = '#{date.future ''365'',''DAYS''}' 36 | ); 37 | 38 | CREATE TABLE users ( 39 | user_id INT PRIMARY KEY, 40 | user_name VARCHAR(255) NOT NULL, 41 | age INT NOT NULL 42 | ) 43 | WITH ( 44 | 'connector' = 'jdbc', 45 | 'url' = 'jdbc:mysql://localhost:3306/mysql-database', 46 | 'table-name' = 'users', 47 | 'username' = 'mysql-user', 48 | 'password' = 'mysql-password' 49 | ); 50 | 51 | SELECT 52 | id AS subscription_id, 53 | type AS subscription_type, 54 | age AS user_age, 55 | CASE 56 | WHEN age < 18 THEN 1 57 | ELSE 0 58 | END AS is_minor 59 | FROM subscriptions usub 60 | JOIN users FOR SYSTEM_TIME AS OF usub.proc_time AS u 61 | ON usub.user_id = u.user_id; 62 | ``` 63 | 64 | ## Example Output 65 | 66 | ![18_lookup_joins](https://user-images.githubusercontent.com/23521087/102645588-fcdeea80-4162-11eb-8581-55a06ea82518.png) 67 | -------------------------------------------------------------------------------- /aggregations-and-analytics/03_group_by_session_window/03_group_by_session_window.md: -------------------------------------------------------------------------------- 1 | # 03 Analyzing Sessions in Time Series Data 2 | 3 | > :bulb: This example will show how to aggregate time-series data in real-time using a `SESSION` window. 4 | 5 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | #### What are Session Windows? 8 | 9 | In a [previous recipe](../01_group_by_window/01_group_by_window.md), you learned about _tumbling windows_. Another way to group time-series data is using [_session windows_](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sql/queries.html#group-windows), which aggregate records into _sessions_ that represent periods of activity followed by gaps of idleness. Think, for example, of user sessions on a website: a user will be active for a given period of time, then leave the website; and each user will be active at different times. To analyze user behaviour, it's useful to aggregate their actions on the website for each period of activity (i.e. _session_). 10 | 11 | Unlike tumbling windows, session windows don't have a fixed duration and are tracked independenlty across keys (i.e. windows of different keys will have different durations). 12 | 13 | #### Using Session Windows 14 | 15 | To count the number of "Forbidden" (403) requests per user over the duration of a session, you can use the `SESSION` built-in group window function. In this example, a session is bounded by a gap of idleness of 10 seconds (`INTERVAL '10' SECOND`). This means that requests that occur within 10 seconds of the last seen request for each user will be merged into the same session window; and any request that occurs outside of this gap will trigger the creation of a new session window. 16 | 17 | > Tip: You can use the `SESSION_START` and `SESSION_ROWTIME` [auxiliary functions](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sql/queries.html#selecting-group-window-start-and-end-timestamps) to check the lower and upper bounds of session windows. 18 | 19 | 20 | ## Script 21 | 22 | ```sql 23 | CREATE TABLE server_logs ( 24 | client_ip STRING, 25 | client_identity STRING, 26 | userid STRING, 27 | log_time TIMESTAMP(3), 28 | request_line STRING, 29 | status_code STRING, 30 | WATERMARK FOR log_time AS log_time - INTERVAL '5' SECONDS 31 | ) WITH ( 32 | 'connector' = 'faker', 33 | 'rows-per-second' = '5', 34 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 35 | 'fields.client_identity.expression' = '-', 36 | 'fields.userid.expression' = '#{regexify ''(morsapaes|knauf|sjwiesman){1}''}', 37 | 'fields.log_time.expression' = '#{date.past ''5'',''SECONDS''}', 38 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 39 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}' 40 | ); 41 | 42 | SELECT 43 | userid, 44 | SESSION_START(log_time, INTERVAL '10' SECOND) AS session_beg, 45 | SESSION_ROWTIME(log_time, INTERVAL '10' SECOND) AS session_end, 46 | COUNT(request_line) AS request_cnt 47 | FROM server_logs 48 | WHERE status_code = '403' 49 | GROUP BY 50 | userid, 51 | SESSION(log_time, INTERVAL '10' SECOND); 52 | ``` 53 | 54 | ## Example Output 55 | 56 | ![03_session_windows](https://user-images.githubusercontent.com/23521087/101628701-7ae31900-3a20-11eb-89c2-231649b7d99f.png) 57 | -------------------------------------------------------------------------------- /other-builtin-functions/01_date_time/01_date_time.md: -------------------------------------------------------------------------------- 1 | # 01 Working with Dates and Timestamps 2 | 3 | > :bulb: This example will show how to use [built-in date and time functions](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/functions/systemFunctions.html#temporal-functions) to manipulate temporal fields. 4 | 5 | The source table (`subscriptions`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | #### Date and Time Functions 8 | 9 | Working with dates and timestamps is commonplace in SQL, but your input may come in different types, formats or even timezones. Flink SQL has multiple built-in functions that are useful to deal with this kind of situation and make it convenient to handle temporal fields. 10 | 11 | Assume you have a table with service subscriptions and that you want to continuously [filter](../../foundations/04_where/04_where.md) these subscriptions to find the ones that have associated payment methods expiring in less than 30 days. The `start_date` and `end_date` are [Unix timestamps](https://en.wikipedia.org/wiki/Unix_time) (i.e. epochs) — which are not very human-readable and should be converted. Also, you want to parse the `payment_expiration` timestamp into its corresponding day, month and year parts. What are some functions that would be useful? 12 | 13 | * `TO_TIMESTAMP(string[, format])`: converts a `STRING` value to a `TIMESTAMP` using the specified format (default: 'yyyy-MM-dd HH:mm:ss') 14 | 15 | * `FROM_UNIXTIME(numeric[, string])`: converts an epoch to a formatted `STRING` (default: 'yyyy-MM-dd HH:mm:ss') 16 | 17 | * `DATE_FORMAT(timestamp, string)`: converts a `TIMESTAMP` to a `STRING` using the specified format 18 | 19 | * `EXTRACT(timeintervalunit FROM temporal)`: returns a `LONG` extracted from the specified date part of a temporal field (e.g. `DAY`,`MONTH`,`YEAR`) 20 | 21 | * `TIMESTAMPDIFF(unit, timepoint1, timepoint2)`: returns the number of time units (`SECOND`, `MINUTE`, `HOUR`, `DAY`, `MONTH` or `YEAR`) between `timepoint1` and `timepoint2` 22 | 23 | * `CURRENT_TIMESTAMP`: returns the current SQL timestamp (UTC) 24 | 25 | For a complete list of built-in date and time functions, check the Flink [documentation](https://ci.apache.org/projects/flink/flink-docs-stable/docs/dev/table/functions/systemfunctions/#temporal-functions). 26 | 27 | > As an exercise, you can try to reproduce the same filtering condition using `TIMESTAMPADD` instead. 28 | 29 | ## Script 30 | 31 | ```sql 32 | CREATE TABLE subscriptions ( 33 | id STRING, 34 | start_date INT, 35 | end_date INT, 36 | payment_expiration TIMESTAMP(3) 37 | ) WITH ( 38 | 'connector' = 'faker', 39 | 'fields.id.expression' = '#{Internet.uuid}', 40 | 'fields.start_date.expression' = '#{number.numberBetween ''1576141834'',''1607764234''}', 41 | 'fields.end_date.expression' = '#{number.numberBetween ''1609060234'',''1639300234''}', 42 | 'fields.payment_expiration.expression' = '#{date.future ''365'',''DAYS''}' 43 | ); 44 | 45 | SELECT 46 | id, 47 | TO_TIMESTAMP(FROM_UNIXTIME(start_date)) AS start_date, 48 | TO_TIMESTAMP(FROM_UNIXTIME(end_date)) AS end_date, 49 | DATE_FORMAT(payment_expiration,'YYYYww') AS exp_yweek, 50 | EXTRACT(DAY FROM payment_expiration) AS exp_day, --same as DAYOFMONTH(ts) 51 | EXTRACT(MONTH FROM payment_expiration) AS exp_month, --same as MONTH(ts) 52 | EXTRACT(YEAR FROM payment_expiration) AS exp_year --same as YEAR(ts) 53 | FROM subscriptions 54 | WHERE 55 | TIMESTAMPDIFF(DAY,CAST(CURRENT_TIMESTAMP AS TIMESTAMP(3)),payment_expiration) < 30; 56 | ``` 57 | 58 | ## Example Output 59 | 60 | ![12_date_time](https://user-images.githubusercontent.com/23521087/101981480-811a0500-3c6d-11eb-9b28-5603d76ba0e6.png) -------------------------------------------------------------------------------- /udfs/01_python_udfs/01_python_udfs.md: -------------------------------------------------------------------------------- 1 | # 01 Extending SQL with Python UDFs 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to extend Flink SQL with custom functions written in Python. 6 | 7 | Flink SQL provides a wide range of [built-in functions](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/functions/systemFunctions.html) that cover most SQL day-to-day work. Sometimes, you need more flexibility to express custom business logic or transformations that aren't easily translatable to SQL: this can be achieved with [User-Defined Functions](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/functions/udfs.html) (UDFs). 8 | 9 | In this example, you'll focus on [Python UDFs](https://ci.apache.org/projects/flink/flink-docs-stable/dev/python/table-api-users-guide/udfs/python_udfs.html) and implement a custom function (`to_fahr`) to convert temperature readings that are continuously generated for different EU and US cities. The Celsius->Fahrenheit conversion should only happen if the city associated with the reading is in the US. 10 | 11 | ## Scripts 12 | 13 | #### Python UDF 14 | 15 | The first step is to create a Python file with the UDF implementation (`python_udf.py`), using Flink's [Python Table API](https://ci.apache.org/projects/flink/flink-docs-stable/dev/python/table-api-users-guide/intro_to_table_api.html). If this is new to you, there are examples on how to write [general](https://ci.apache.org/projects/flink/flink-docs-stable/dev/python/table-api-users-guide/udfs/python_udfs.html) and [vectorized](https://ci.apache.org/projects/flink/flink-docs-stable/dev/python/table-api-users-guide/udfs/vectorized_python_udfs.html) Python UDFs in the Flink documentation. 16 | 17 | ```python 18 | from pyflink.table import DataTypes 19 | from pyflink.table.udf import udf 20 | 21 | us_cities = {"Chicago","Portland","Seattle","New York"} 22 | 23 | @udf(input_types=[DataTypes.STRING(), DataTypes.FLOAT()], 24 | result_type=DataTypes.FLOAT()) 25 | def to_fahr(city, temperature): 26 | 27 | if city in us_cities: 28 | 29 | fahr = ((temperature * 9.0 / 5.0) + 32.0) 30 | 31 | return fahr 32 | else: 33 | return temperature 34 | ``` 35 | 36 | For detailed instructions on how to then make the Python file available as a UDF in the SQL Client, please refer to [this documentation page](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sqlClient.html#user-defined-functions). 37 | 38 | #### SQL 39 | 40 | The source table (`temperature_measurements`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 41 | 42 | ```sql 43 | --Register the Python UDF using the fully qualified 44 | --name of the function ([module name].[object name]) 45 | CREATE FUNCTION to_fahr AS 'python_udf.to_fahr' 46 | LANGUAGE PYTHON; 47 | 48 | 49 | CREATE TABLE temperature_measurements ( 50 | city STRING, 51 | temperature FLOAT, 52 | measurement_time TIMESTAMP(3), 53 | WATERMARK FOR measurement_time AS measurement_time - INTERVAL '15' SECONDS 54 | ) 55 | WITH ( 56 | 'connector' = 'faker', 57 | 'fields.temperature.expression' = '#{number.numberBetween ''0'',''42''}', 58 | 'fields.measurement_time.expression' = '#{date.past ''15'',''SECONDS''}', 59 | 'fields.city.expression' = '#{regexify ''(Copenhagen|Berlin|Chicago|Portland|Seattle|New York){1}''}' 60 | ); 61 | 62 | 63 | --Use to_fahr() to convert temperatures in US cities from C to F 64 | SELECT city, 65 | temperature AS tmp, 66 | to_fahr(city,temperature) AS tmp_conv, 67 | measurement_time 68 | FROM temperature_measurements; 69 | ``` 70 | 71 | ## Example Output 72 | 73 | ![01_python_udfs](https://user-images.githubusercontent.com/23521087/106733744-8ca4ff00-6612-11eb-9721-e4a74fb07329.gif) -------------------------------------------------------------------------------- /foundations/06_order_by/06_order_by.md: -------------------------------------------------------------------------------- 1 | # 06 Sorting Tables 2 | 3 | > :bulb: This example will show how you can sort a table, particularly unbounded tables. 4 | 5 | Flink SQL supports `ORDER BY`. 6 | Bounded Tables can be sorted by any column, descending or ascending. 7 | 8 | To use `ORDER BY` on unbounded tables like `server_logs` the primary sorting key needs to be a [time attribute](https://docs.ververica.com/user_guide/sql_development/table_view.html#time-attributes) like `log_time`. 9 | 10 | In first example below, we are sorting the `server_logs` by `log_time`. 11 | The second example is a bit more advanced: 12 | It sorts the number of logs per minute and browser by the `window_time` (a time attribute) and the `cnt_browser` (descending), so that the browser with the highest number of logs is at the top of each window. 13 | 14 | ## Script 15 | 16 | ```sql 17 | CREATE TEMPORARY TABLE server_logs ( 18 | client_ip STRING, 19 | client_identity STRING, 20 | userid STRING, 21 | user_agent STRING, 22 | log_time TIMESTAMP(3), 23 | request_line STRING, 24 | status_code STRING, 25 | size INT, 26 | WATERMARK FOR log_time AS log_time - INTERVAL '15' SECONDS 27 | ) WITH ( 28 | 'connector' = 'faker', 29 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 30 | 'fields.client_identity.expression' = '-', 31 | 'fields.userid.expression' = '-', 32 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 33 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 34 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 35 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 36 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 37 | ); 38 | 39 | SELECT * FROM server_logs 40 | ORDER BY log_time; 41 | ``` 42 | 43 | ## Example Output 44 | 45 | ![06_order_by](https://user-images.githubusercontent.com/23521087/105504299-e24ee280-5cc7-11eb-8935-ed203e604f8d.png) 46 | 47 | ## Advanced Example 48 | 49 |
50 | Advanced Example 51 | 52 | ### Script 53 | 54 | ```sql 55 | CREATE TEMPORARY TABLE server_logs ( 56 | client_ip STRING, 57 | client_identity STRING, 58 | userid STRING, 59 | user_agent STRING, 60 | log_time TIMESTAMP(3), 61 | request_line STRING, 62 | status_code STRING, 63 | size INT, 64 | WATERMARK FOR log_time AS log_time - INTERVAL '15' SECONDS 65 | ) WITH ( 66 | 'connector' = 'faker', 67 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 68 | 'fields.client_identity.expression' = '-', 69 | 'fields.userid.expression' = '-', 70 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 71 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 72 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 73 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 74 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 75 | ); 76 | 77 | SELECT 78 | TUMBLE_ROWTIME(log_time, INTERVAL '1' MINUTE) AS window_time, 79 | REGEXP_EXTRACT(user_agent,'[^\/]+') AS browser, 80 | COUNT(*) AS cnt_browser 81 | FROM server_logs 82 | GROUP BY 83 | REGEXP_EXTRACT(user_agent,'[^\/]+'), 84 | TUMBLE(log_time, INTERVAL '1' MINUTE) 85 | ORDER BY 86 | window_time, 87 | cnt_browser DESC; 88 | ``` 89 | 90 | ### Example Output 91 | 92 | ![06_order_by_advanced](https://user-images.githubusercontent.com/23521087/105504249-d5ca8a00-5cc7-11eb-984a-e1eaf6622995.png) 93 | 94 |
95 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Apache Flink SQL Cookbook 2 | 3 | The [Apache Flink SQL](https://docs.ververica.com/user_guide/sql_development/index.html) Cookbook is a curated collection of examples, patterns, and use cases of Apache Flink SQL. 4 | Many of the recipes are completely self-contained and can be run in [Ververica Platform](https://docs.ververica.com/index.html) as is. 5 | 6 | The cookbook is a living document. :seedling: 7 | 8 | ## Table of Contents 9 | 10 | ### Foundations 11 | 12 | 1. [Creating Tables](foundations/01_create_table/01_create_table.md) 13 | 2. [Inserting Into Tables](foundations/02_insert_into/02_insert_into.md) 14 | 3. [Working with Temporary Tables](foundations/03_temporary_table/03_temporary_table.md) 15 | 4. [Filtering Data](foundations/04_where/04_where.md) 16 | 5. [Aggregating Data](foundations/05_group_by/05_group_by.md) 17 | 6. [Sorting Tables](foundations/06_order_by/06_order_by.md) 18 | 7. [Encapsulating Logic with (Temporary) Views](foundations/07_views/07_views.md) 19 | 8. [Writing Results into Multiple Tables](foundations/08_statement_sets/08_statement_sets.md) 20 | 9. [Convert timestamps with timezones](foundations/09_convert_timezones/09_convert_timezones.md) 21 | 22 | ### Aggregations and Analytics 23 | 1. [Aggregating Time Series Data](aggregations-and-analytics/01_group_by_window/01_group_by_window_tvf.md) 24 | 2. [Watermarks](aggregations-and-analytics/02_watermarks/02_watermarks.md) 25 | 3. [Analyzing Sessions in Time Series Data](aggregations-and-analytics/03_group_by_session_window/03_group_by_session_window.md) 26 | 4. [Rolling Aggregations on Time Series Data](aggregations-and-analytics/04_over/04_over.md) 27 | 5. [Continuous Top-N](aggregations-and-analytics/05_top_n/05_top_n.md) 28 | 6. [Deduplication](aggregations-and-analytics/06_dedup/06_dedup.md) 29 | 7. [Chained (Event) Time Windows](aggregations-and-analytics/07_chained_windows/07_chained_windows.md) 30 | 8. [Detecting Patterns with MATCH_RECOGNIZE](aggregations-and-analytics/08_match_recognize/08_match_recognize.md) 31 | 9. [Maintaining Materialized Views with Change Data Capture (CDC) and Debezium](aggregations-and-analytics/09_cdc_materialized_view/09_cdc_materialized_view.md) 32 | 10. [Hopping Time Windows](aggregations-and-analytics/10_hopping_time_windows/10_hopping_time_windows.md) 33 | 11. [Window Top-N](aggregations-and-analytics/11_window_top_n/11_window_top_n.md) 34 | 12. [Retrieve previous row value without self-join](aggregations-and-analytics/12_lag/12_lag.md) 35 | 36 | ### Other Built-in Functions & Operators 37 | 1. [Working with Dates and Timestamps](other-builtin-functions/01_date_time/01_date_time.md) 38 | 2. [Building the Union of Multiple Streams](other-builtin-functions/02_union-all/02_union-all.md) 39 | 3. [Filtering out Late Data](other-builtin-functions/03_current_watermark/03_current_watermark.md) 40 | 4. [Overriding table options](other-builtin-functions/04_override_table_options/04_override_table_options.md) 41 | 5. [Expanding arrays into new rows](other-builtin-functions/05_expanding_arrays/05_expanding_arrays.md) 42 | 6. [Split strings into maps](other-builtin-functions/06_split_strings_into_maps/06_split_strings_into_maps.md) 43 | 44 | ### User-Defined Functions (UDFs) 45 | 1. [Extending SQL with Python UDFs](udfs/01_python_udfs/01_python_udfs.md) 46 | 47 | ### Joins 48 | 49 | 1. [Regular Joins](joins/01_regular_joins/01_regular_joins.md) 50 | 2. [Interval Joins](joins/02_interval_joins/02_interval_joins.md) 51 | 3. [Temporal Table Join between a non-compacted and compacted Kafka Topic](joins/03_kafka_join/03_kafka_join.md) 52 | 4. [Lookup Joins](joins/04_lookup_joins/04_lookup_joins.md) 53 | 5. [Star Schema Denormalization (N-Way Join)](joins/05_star_schema/05_star_schema.md) 54 | 6. [Lateral Table Join](joins/06_lateral_join/06_lateral_join.md) 55 | 56 | ### Former Recipes 57 | 1. [Aggregating Time Series Data (Before Flink 1.13)](aggregations-and-analytics/01_group_by_window/01_group_by_window.md) 58 | 59 | ## About Apache Flink 60 | 61 | Apache Flink is an open source stream processing framework with powerful stream- and batch-processing capabilities. 62 | 63 | Learn more about Flink at https://flink.apache.org/. 64 | 65 | ## License 66 | 67 | Copyright © 2020-2022 Ververica GmbH 68 | 69 | Distributed under Apache License, Version 2.0. 70 | -------------------------------------------------------------------------------- /other-builtin-functions/03_current_watermark/03_current_watermark.md: -------------------------------------------------------------------------------- 1 | # 03 Filtering out Late Data 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.14%2B-lightgrey) 4 | 5 | > :bulb: This example will show how to filter out late data using the `CURRENT_WATERMARK` function. 6 | 7 | The source table (`mobile_usage`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 8 | 9 | As explained before in the [watermarks recipe](../../aggregations-and-analytics/02_watermarks/02_watermarks.md), Flink uses watermarks to measure progress in event time. By using a `WATERMARK` attribute in a table's DDL, we signify a column as the table's event time attribute and tell Flink how out of order we expect our data to arrive. 10 | 11 | There are many cases when rows are arriving even more out of order than anticipated, i.e. after the watermark. This data is called *late*. An example could be when someone is using a mobile app while being offline because of lack of mobile coverage or flight mode being enabled. When Internet access is restored, previously tracked activities would then be sent. 12 | 13 | In this recipe, we'll filter out this late data using the [`CURRENT_WATERMARK`](https://ci.apache.org/projects/flink/flink-docs-release-1.14/docs/dev/table/functions/systemfunctions/) function. In the first statement, we'll use the non-late data combined with the [`TUMBLE`](../../aggregations-and-analytics/01_group_by_window/01_group_by_window_tvf.md) function to send the unique IP addresses per minute to a downstream consumer (like a BI tool). Next to this use case, we're sending the late data to a different sink. For example, you might want to use these rows to change the results of your product recommender for offline mobile app users. 14 | 15 | This table DDL contains both an event time and a processing time definition. `ingest_time` is defined as processing time, while `log_time` is defined as event time and will contain timestamps between 45 and 10 seconds ago. 16 | 17 | ## Script 18 | 19 | ```sql 20 | -- Create source table 21 | CREATE TABLE `mobile_usage` ( 22 | `activity` STRING, 23 | `client_ip` STRING, 24 | `ingest_time` AS PROCTIME(), 25 | `log_time` TIMESTAMP_LTZ(3), 26 | WATERMARK FOR log_time AS log_time - INTERVAL '15' SECONDS 27 | ) WITH ( 28 | 'connector' = 'faker', 29 | 'rows-per-second' = '50', 30 | 'fields.activity.expression' = '#{regexify ''(open_push_message|discard_push_message|open_app|display_overview|change_settings)''}', 31 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 32 | 'fields.log_time.expression' = '#{date.past ''45'',''10'',''SECONDS''}' 33 | ); 34 | 35 | -- Create sink table for rows that are non-late 36 | CREATE TABLE `unique_users_per_window` ( 37 | `window_start` TIMESTAMP(3), 38 | `window_end` TIMESTAMP(3), 39 | `ip_addresses` BIGINT 40 | ) WITH ( 41 | 'connector' = 'blackhole' 42 | ); 43 | 44 | -- Create sink table for rows that are late 45 | CREATE TABLE `late_usage_events` ( 46 | `activity` STRING, 47 | `client_ip` STRING, 48 | `ingest_time` TIMESTAMP_LTZ(3), 49 | `log_time` TIMESTAMP_LTZ(3), 50 | `current_watermark` TIMESTAMP_LTZ(3) 51 | ) WITH ( 52 | 'connector' = 'blackhole' 53 | ); 54 | 55 | -- Create a view with non-late data 56 | CREATE TEMPORARY VIEW `mobile_data` AS 57 | SELECT * FROM mobile_usage 58 | WHERE CURRENT_WATERMARK(log_time) IS NULL 59 | OR log_time > CURRENT_WATERMARK(log_time); 60 | 61 | -- Create a view with late data 62 | CREATE TEMPORARY VIEW `late_mobile_data` AS 63 | SELECT * FROM mobile_usage 64 | WHERE CURRENT_WATERMARK(log_time) IS NOT NULL 65 | AND log_time <= CURRENT_WATERMARK(log_time); 66 | 67 | BEGIN STATEMENT SET; 68 | 69 | -- Send all rows that are non-late to the sink for data that's on time 70 | INSERT INTO `unique_users_per_window` 71 | SELECT `window_start`, `window_end`, COUNT(DISTINCT client_ip) AS `ip_addresses` 72 | FROM TABLE( 73 | TUMBLE(TABLE mobile_data, DESCRIPTOR(log_time), INTERVAL '1' MINUTE)) 74 | GROUP BY window_start, window_end; 75 | 76 | -- Send all rows that are late to the sink for late data 77 | INSERT INTO `late_usage_events` 78 | SELECT *, CURRENT_WATERMARK(log_time) as `current_watermark` from `late_mobile_data`; 79 | 80 | END; 81 | ``` 82 | 83 | ## Example Output 84 | 85 | ![03_current_watermark](03_current_watermark.png) 86 | 87 | ### Late data 88 | 89 | ![03_late_data](03_late_data.png) 90 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/09_cdc_materialized_view.md: -------------------------------------------------------------------------------- 1 | # 09 Maintaining Materialized Views with Change Data Capture (CDC) and Debezium 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.11%2B-lightgrey) 4 | 5 | > :bulb: This example will show how you can use Flink SQL and Debezium to maintain a materialized view based on database changelog streams. 6 | 7 | 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? 8 | 9 | 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). 10 | 11 | #### Let's get to it! 12 | 13 | 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). 14 | 15 | ## Pre-requisites 16 | 17 | 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: 18 | 19 | To get the setup up and running, run: 20 | 21 | `docker-compose build` 22 | 23 | `docker-compose up -d` 24 | 25 | Once all the services are up, you can start the Flink SQL client: 26 | 27 | `docker-compose exec sql-client ./sql-client.sh` 28 | 29 | ## How it works 30 | 31 | 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. 32 | 33 | ```sql 34 | CREATE TABLE accident_claims ( 35 | claim_id INT, 36 | claim_total FLOAT, 37 | claim_total_receipt VARCHAR(50), 38 | claim_currency VARCHAR(3), 39 | member_id INT, 40 | accident_date VARCHAR(20), 41 | accident_type VARCHAR(20), 42 | accident_detail VARCHAR(20), 43 | claim_date VARCHAR(20), 44 | claim_status VARCHAR(10), 45 | ts_created VARCHAR(20), 46 | ts_updated VARCHAR(20) 47 | ) WITH ( 48 | 'connector' = 'postgres-cdc', 49 | 'hostname' = 'postgres', 50 | 'port' = '5432', 51 | 'username' = 'postgres', 52 | 'password' = 'postgres', 53 | 'database-name' = 'postgres', 54 | 'schema-name' = 'claims', 55 | 'table-name' = 'accident_claims' 56 | ); 57 | ``` 58 | 59 | 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`): 60 | 61 | ```sql 62 | SELECT accident_detail, 63 | SUM(claim_total) AS agg_claim_costs 64 | FROM accident_claims 65 | WHERE claim_status <> 'DENIED' 66 | GROUP BY accident_detail; 67 | ``` 68 | 69 | 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: 70 | 71 | `cat ./postgres_datagen.sql | docker exec -i flink-cdc-postgres psql -U postgres -d postgres` 72 | 73 | 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). 74 | 75 | ## Example Output 76 | 77 | ![09_cdc_materialized_view](https://user-images.githubusercontent.com/23521087/109818653-81ee8180-7c33-11eb-9a76-b1004de8fe23.gif) 78 | -------------------------------------------------------------------------------- /aggregations-and-analytics/08_match_recognize/08_match_recognize.md: -------------------------------------------------------------------------------- 1 | # 08 Detecting patterns with MATCH_RECOGNIZE 2 | 3 | > :bulb: This example will show how you can use Flink SQL to detect patterns in a stream of events with `MATCH_RECOGNIZE`. 4 | 5 | A common (but historically complex) task in SQL day-to-day work is to identify meaningful sequences of events in a data set — also known as Complex Event Processing (CEP). This becomes even more relevant when dealing with streaming data, as you want to react quickly to known patterns or changing trends to deliver up-to-date business insights. In Flink SQL, you can easily perform this kind of tasks using the standard SQL clause [`MATCH_RECOGNIZE`](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/streaming/match_recognize.html). 6 | 7 | ## Breaking down MATCH_RECOGNIZE 8 | 9 | In this example, you want to find users that downgraded their service subscription from one of the premium tiers (`type IN ('premium','platinum')`) to the basic tier. 10 | 11 | #### Input 12 | 13 | The input argument of `MATCH_RECOGNIZE` will be a row pattern table based on `subscriptions`. As a first step, logical partitioning and ordering must be applied to the input row pattern table to ensure that event processing is correct and deterministic: 14 | 15 | ```sql 16 | PARTITION BY user_id 17 | ORDER BY proc_time 18 | ``` 19 | 20 | #### Output 21 | 22 | Row pattern columns are then defined in the `MEASURES` clause, which can be thought of as the `SELECT` of `MATCH_RECOGNIZE`. If you're interested in getting the type of premium subscription associated with the last event before the downgrade, you can fetch it using the [logical offset](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/streaming/match_recognize.html#logical-offsets) operator `LAST`. The downgrade date can be extrapolated from the `start_date` of the first basic subscription event following any existing premium one(s). 23 | 24 | ```sql 25 | MEASURES 26 | LAST(PREMIUM.type) AS premium_type, 27 | AVG(TIMESTAMPDIFF(DAY,PREMIUM.start_date,PREMIUM.end_date)) AS premium_avg_duration, 28 | BASIC.start_date AS downgrade_date 29 | AFTER MATCH SKIP PAST LAST ROW 30 | ``` 31 | 32 | #### Pattern Definition 33 | 34 | Patterns are specified in the `PATTERN` clause using row-pattern variables (i.e. event types) and regular expressions. These variables must also be associated with the matching conditions that events must meet to be included in the pattern, using the `DEFINE` clause. Here, you are interested in matching one or more premium subscription events (`PREMIUM+`) followed by a basic subscription event (`BASIC`): 35 | 36 | ```sql 37 | PATTERN (PREMIUM+ BASIC) 38 | DEFINE PREMIUM AS PREMIUM.type IN ('premium','platinum'), 39 | BASIC AS BASIC.type = 'basic'); 40 | ``` 41 | 42 | ## Script 43 | 44 | The source table (`subscriptions`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 45 | 46 | ```sql 47 | CREATE TABLE subscriptions ( 48 | id STRING, 49 | user_id INT, 50 | type STRING, 51 | start_date TIMESTAMP(3), 52 | end_date TIMESTAMP(3), 53 | payment_expiration TIMESTAMP(3), 54 | proc_time AS PROCTIME() 55 | ) WITH ( 56 | 'connector' = 'faker', 57 | 'fields.id.expression' = '#{Internet.uuid}', 58 | 'fields.user_id.expression' = '#{number.numberBetween ''1'',''50''}', 59 | 'fields.type.expression'= '#{regexify ''(basic|premium|platinum){1}''}', 60 | 'fields.start_date.expression' = '#{date.past ''30'',''DAYS''}', 61 | 'fields.end_date.expression' = '#{date.future ''15'',''DAYS''}', 62 | 'fields.payment_expiration.expression' = '#{date.future ''365'',''DAYS''}' 63 | ); 64 | 65 | SELECT * 66 | FROM subscriptions 67 | MATCH_RECOGNIZE (PARTITION BY user_id 68 | ORDER BY proc_time 69 | MEASURES 70 | LAST(PREMIUM.type) AS premium_type, 71 | AVG(TIMESTAMPDIFF(DAY,PREMIUM.start_date,PREMIUM.end_date)) AS premium_avg_duration, 72 | BASIC.start_date AS downgrade_date 73 | AFTER MATCH SKIP PAST LAST ROW 74 | --Pattern: one or more 'premium' or 'platinum' subscription events (PREMIUM) 75 | --followed by a 'basic' subscription event (BASIC) for the same `user_id` 76 | PATTERN (PREMIUM+ BASIC) 77 | DEFINE PREMIUM AS PREMIUM.type IN ('premium','platinum'), 78 | BASIC AS BASIC.type = 'basic'); 79 | ``` 80 | 81 | ## Example Output 82 | 83 | ![23_match_recognize](https://user-images.githubusercontent.com/2392216/108039085-ee665f80-703b-11eb-93f9-f8e3b684f315.png) 84 | -------------------------------------------------------------------------------- /foundations/08_statement_sets/08_statement_sets.md: -------------------------------------------------------------------------------- 1 | # 08 Writing Results into Multiple Tables 2 | 3 | ![Twitter Badge](https://img.shields.io/badge/Flink%20Version-1.13%2B-lightgrey) 4 | 5 | > :bulb: In this recipe, you will learn how to use [Statement Sets](https://docs.ververica.com/user_guide/sql_development/sql_scripts.html#sql-statements) to run multiple `INSERT INTO` statements in a single, optimized Flink Job. 6 | 7 | Many product requirements involve outputting the results of a streaming application to two or more sinks, such as [Apache Kafka](https://docs.ververica.com/user_guide/sql_development/connectors.html#apache-kafka) for real-time use cases, or a [Filesystem](https://docs.ververica.com/user_guide/sql_development/connectors.html#file-system) for offline ones. 8 | Other times, two queries are not the same but share some extensive intermediate operations. 9 | 10 | When working with server logs, the support team would like to see the number of status codes per browser every 5 minutes to have real-time insights into a web pages' status. 11 | Additionally, they would like the same information on an hourly basis made available as partitioned [Apache Parquet](https://docs.ververica.com/user_guide/sql_development/connectors.html#apache-parquet) files so they can perform historical analysis. 12 | 13 | We could quickly write two Flink SQL queries to solve both these requirements, but that would not be efficient. 14 | These queries have a lot of duplicated work, like reading the source logs Kafka topic and cleansing the data. 15 | 16 | Ververica Platform includes a feature called `STATEMENT SET`s, that allows for multiplexing `INSERT INTO` statements into a single query holistically optimized by Apache Flink and deployed as a single application. 17 | 18 | ```sql 19 | CREATE TEMPORARY TABLE server_logs ( 20 | client_ip STRING, 21 | client_identity STRING, 22 | userid STRING, 23 | user_agent STRING, 24 | log_time TIMESTAMP(3), 25 | request_line STRING, 26 | status_code STRING, 27 | size INT, 28 | WATERMARK FOR log_time AS log_time - INTERVAL '30' SECONDS 29 | ) WITH ( 30 | 'connector' = 'faker', 31 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 32 | 'fields.client_identity.expression' = '-', 33 | 'fields.userid.expression' = '-', 34 | 'fields.user_agent.expression' = '#{Internet.userAgentAny}', 35 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 36 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 37 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 38 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 39 | ); 40 | 41 | CREATE TEMPORARY TABLE realtime_aggregations ( 42 | `browser` STRING, 43 | `status_code` STRING, 44 | `end_time` TIMESTAMP(3), 45 | `requests` BIGINT NOT NULL 46 | ) WITH ( 47 | 'connector' = 'kafka', 48 | 'topic' = 'browser-status-codes', 49 | 'properties.bootstrap.servers' = 'localhost:9092', 50 | 'properties.group.id' = 'browser-countds', 51 | 'format' = 'avro' 52 | ); 53 | 54 | 55 | CREATE TEMPORARY TABLE offline_datawarehouse ( 56 | `browser` STRING, 57 | `status_code` STRING, 58 | `dt` STRING, 59 | `hour` STRING, 60 | `requests` BIGINT NOT NULL 61 | ) PARTITIONED BY (`dt`, `hour`) WITH ( 62 | 'connector' = 'filesystem', 63 | 'path' = 's3://my-bucket/browser-into', 64 | 'sink.partition-commit.trigger' = 'partition-time', 65 | 'format' = 'parquet' 66 | ); 67 | 68 | -- This is a shared view that will be used by both 69 | -- insert into statements 70 | CREATE TEMPORARY VIEW browsers AS 71 | SELECT 72 | REGEXP_EXTRACT(user_agent,'[^\/]+') AS browser, 73 | status_code, 74 | log_time 75 | FROM server_logs; 76 | 77 | BEGIN STATEMENT SET; 78 | INSERT INTO realtime_aggregations 79 | SELECT 80 | browser, 81 | status_code, 82 | TUMBLE_ROWTIME(log_time, INTERVAL '5' MINUTE) AS end_time, 83 | COUNT(*) requests 84 | FROM browsers 85 | GROUP BY 86 | browser, 87 | status_code, 88 | TUMBLE(log_time, INTERVAL '5' MINUTE); 89 | INSERT INTO offline_datawarehouse 90 | SELECT 91 | browser, 92 | status_code, 93 | DATE_FORMAT(TUMBLE_ROWTIME(log_time, INTERVAL '1' HOUR), 'yyyy-MM-dd') AS `dt`, 94 | DATE_FORMAT(TUMBLE_ROWTIME(log_time, INTERVAL '1' HOUR), 'HH') AS `hour`, 95 | COUNT(*) requests 96 | FROM browsers 97 | GROUP BY 98 | browser, 99 | status_code, 100 | TUMBLE(log_time, INTERVAL '1' HOUR); 101 | END; 102 | ``` 103 | 104 | Looking at the deployed Job Graph, we can see Flink SQL only performs the shared computation once to achieve the most cost and resource-efficient execution of our query! 105 | 106 | ![08_jobgraph](https://user-images.githubusercontent.com/23521087/105504375-fb579380-5cc7-11eb-888e-12a1ce7d6f50.png) 107 | -------------------------------------------------------------------------------- /joins/03_kafka_join/03_kafka_join.md: -------------------------------------------------------------------------------- 1 | # 03 Temporal Table Join between a non-compacted and compacted Kafka Topic 2 | 3 | > :bulb: In this recipe, you will see how to correctly enrich records from one Kafka topic with the corresponding records of another Kafka topic when the order of events matters. 4 | 5 | Temporal table joins take an arbitrary table (left input/probe site) and correlate each row to the corresponding row’s relevant version in a versioned table (right input/build side). 6 | Flink uses the SQL syntax of ``FOR SYSTEM_TIME AS OF`` to perform this operation. 7 | 8 | In this recipe, we want join each transaction (`transactions`) to its correct currency rate (`currency_rates`, a versioned table) **as of the time when the transaction happened**. 9 | A similar example would be to join each order with the customer details as of the time when the order happened. 10 | This is exactly what an event-time temporal table join does. 11 | A temporal table join in Flink SQL provides correct, deterministic results in the presence of out-of-orderness and arbitrary time skew between the two tables. 12 | 13 | Both the `transactions` and `currency_rates` tables are backed by Kafka topics, but in the case of rates this topic is compacted (i.e. only the most recent messages for a given key are kept as updated rates flow in). 14 | Records in `transactions` are interpreted as inserts only, and so the table is backed by the [standard Kafka connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/kafka.html) (`connector` = `kafka`); while the records in `currency_rates` need to be interpreted as upserts based on a primary key, which requires the [Upsert Kafka connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/upsert-kafka.html) (`connector` = `upsert-kafka`). 15 | 16 | ## Script 17 | 18 | ```sql 19 | CREATE TEMPORARY TABLE currency_rates ( 20 | `currency_code` STRING, 21 | `eur_rate` DECIMAL(6,4), 22 | `rate_time` TIMESTAMP(3), 23 | WATERMARK FOR `rate_time` AS rate_time - INTERVAL '15' SECOND, 24 | PRIMARY KEY (currency_code) NOT ENFORCED 25 | ) WITH ( 26 | 'connector' = 'upsert-kafka', 27 | 'topic' = 'currency_rates', 28 | 'properties.bootstrap.servers' = 'localhost:9092', 29 | 'key.format' = 'raw', 30 | 'value.format' = 'json' 31 | ); 32 | 33 | CREATE TEMPORARY TABLE transactions ( 34 | `id` STRING, 35 | `currency_code` STRING, 36 | `total` DECIMAL(10,2), 37 | `transaction_time` TIMESTAMP(3), 38 | WATERMARK FOR `transaction_time` AS transaction_time - INTERVAL '30' SECOND 39 | ) WITH ( 40 | 'connector' = 'kafka', 41 | 'topic' = 'transactions', 42 | 'properties.bootstrap.servers' = 'localhost:9092', 43 | 'key.format' = 'raw', 44 | 'key.fields' = 'id', 45 | 'value.format' = 'json', 46 | 'value.fields-include' = 'ALL' 47 | ); 48 | 49 | SELECT 50 | t.id, 51 | t.total * c.eur_rate AS total_eur, 52 | t.total, 53 | c.currency_code, 54 | t.transaction_time 55 | FROM transactions t 56 | JOIN currency_rates FOR SYSTEM_TIME AS OF t.transaction_time AS c 57 | ON t.currency_code = c.currency_code; 58 | ``` 59 | 60 | ## Example Output 61 | 62 | ![kafka_join](https://user-images.githubusercontent.com/11538663/102418338-d2bfe800-3ffd-11eb-995a-13fa116b538f.gif) 63 | 64 | ## Data Generators 65 | 66 |
67 | Data Generators 68 | 69 | The two topics are populated using a Flink SQL job, too. 70 | We use the [`faker` connector](https://flink-packages.org/packages/flink-faker) to generate rows in memory based on Java Faker expressions and write those to the respective Kafka topics. 71 | 72 | ### ``currency_rates`` Topic 73 | 74 | ### Script 75 | 76 | ```sql 77 | CREATE TEMPORARY TABLE currency_rates_faker 78 | WITH ( 79 | 'connector' = 'faker', 80 | 'fields.currency_code.expression' = '#{Currency.code}', 81 | 'fields.eur_rate.expression' = '#{Number.randomDouble ''4'',''0'',''10''}', 82 | 'fields.rate_time.expression' = '#{date.past ''15'',''SECONDS''}', 83 | 'rows-per-second' = '100' 84 | ) LIKE currency_rates (EXCLUDING OPTIONS); 85 | 86 | INSERT INTO currency_rates SELECT * FROM currency_rates_faker; 87 | ``` 88 | #### Kafka Topic 89 | 90 | ```shell script 91 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic currency_rates --property print.key=true --property key.separator=" - " 92 | HTG - {"currency_code":"HTG","eur_rate":0.0136,"rate_time":"2020-12-16 22:22:02"} 93 | BZD - {"currency_code":"BZD","eur_rate":1.6545,"rate_time":"2020-12-16 22:22:03"} 94 | BZD - {"currency_code":"BZD","eur_rate":3.616,"rate_time":"2020-12-16 22:22:10"} 95 | BHD - {"currency_code":"BHD","eur_rate":4.5308,"rate_time":"2020-12-16 22:22:05"} 96 | KHR - {"currency_code":"KHR","eur_rate":1.335,"rate_time":"2020-12-16 22:22:06"} 97 | ``` 98 | 99 | ### ``transactions`` Topic 100 | 101 | #### Script 102 | 103 | ```sql 104 | CREATE TEMPORARY TABLE transactions_faker 105 | WITH ( 106 | 'connector' = 'faker', 107 | 'fields.id.expression' = '#{Internet.UUID}', 108 | 'fields.currency_code.expression' = '#{Currency.code}', 109 | 'fields.total.expression' = '#{Number.randomDouble ''2'',''10'',''1000''}', 110 | 'fields.transaction_time.expression' = '#{date.past ''30'',''SECONDS''}', 111 | 'rows-per-second' = '100' 112 | ) LIKE transactions (EXCLUDING OPTIONS); 113 | 114 | INSERT INTO transactions SELECT * FROM transactions_faker; 115 | ``` 116 | 117 | #### Kafka Topic 118 | 119 | ```shell script 120 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic transactions --property print.key=true --property key.separator=" - " 121 | e102e91f-47b9-434e-86e1-34fb1196d91d - {"id":"e102e91f-47b9-434e-86e1-34fb1196d91d","currency_code":"SGD","total":494.07,"transaction_time":"2020-12-16 22:18:46"} 122 | bf028363-5ee4-4a5a-9068-b08392d59f0b - {"id":"bf028363-5ee4-4a5a-9068-b08392d59f0b","currency_code":"EEK","total":906.8,"transaction_time":"2020-12-16 22:18:46"} 123 | e22374b5-82da-4c6d-b4c6-f27a818a58ab - {"id":"e22374b5-82da-4c6d-b4c6-f27a818a58ab","currency_code":"GYD","total":80.66,"transaction_time":"2020-12-16 22:19:02"} 124 | 81b2ce89-26c2-4df3-b12a-8ca921902ac4 - {"id":"81b2ce89-26c2-4df3-b12a-8ca921902ac4","currency_code":"EGP","total":521.98,"transaction_time":"2020-12-16 22:18:57"} 125 | 53c4fd3f-af6e-41d3-a677-536f4c86e010 - {"id":"53c4fd3f-af6e-41d3-a677-536f4c86e010","currency_code":"UYU","total":936.26,"transaction_time":"2020-12-16 22:18:59"} 126 | ``` 127 | 128 |
129 | -------------------------------------------------------------------------------- /aggregations-and-analytics/07_chained_windows/07_chained_windows.md: -------------------------------------------------------------------------------- 1 | # 07 Chained (Event) Time Windows 2 | 3 | > :bulb: This example will show how to efficiently aggregate time series data on two different levels of granularity. 4 | 5 | The source table (`server_logs`) is backed by the [`faker` connector](https://flink-packages.org/packages/flink-faker), which continuously generates rows in memory based on Java Faker expressions. 6 | 7 | Based on our `server_logs` table we would like to compute the average request size over one minute **as well as five minute (event) windows.** 8 | For this, you could run two queries, similar to the one in [Aggregating Time Series Data](../01_group_by_window/01_group_by_window.md). 9 | At the end of the page is the script and resulting JobGraph from this approach. 10 | 11 | In the main part, we will follow a slightly more efficient approach that chains the two aggregations: the one-minute aggregation output serves as the five-minute aggregation input. 12 | 13 | We then use a [Statements Set](../../foundations/08_statement_sets/08_statement_sets.md) to write out the two result tables. 14 | To keep this example self-contained, we use two tables of type `blackhole` instead of `kafka`, `filesystem`, or any other [connectors](https://ci.apache.org/projects/flink/flink-docs-stable/docs/connectors/table/overview/). 15 | 16 | ## Script 17 | 18 | ```sql 19 | CREATE TEMPORARY TABLE server_logs ( 20 | log_time TIMESTAMP(3), 21 | client_ip STRING, 22 | client_identity STRING, 23 | userid STRING, 24 | request_line STRING, 25 | status_code STRING, 26 | size INT, 27 | WATERMARK FOR log_time AS log_time - INTERVAL '15' SECONDS 28 | ) WITH ( 29 | 'connector' = 'faker', 30 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 31 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 32 | 'fields.client_identity.expression' = '-', 33 | 'fields.userid.expression' = '-', 34 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 35 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 36 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 37 | ); 38 | 39 | CREATE TEMPORARY TABLE avg_request_size_1m ( 40 | window_start TIMESTAMP(3), 41 | window_end TIMESTAMP(3), 42 | avg_size BIGINT 43 | ) 44 | WITH ( 45 | 'connector' = 'blackhole' 46 | ); 47 | 48 | CREATE TEMPORARY TABLE avg_request_size_5m ( 49 | window_start TIMESTAMP(3), 50 | window_end TIMESTAMP(3), 51 | avg_size BIGINT 52 | ) 53 | WITH ( 54 | 'connector' = 'blackhole' 55 | ); 56 | 57 | CREATE TEMPORARY VIEW server_logs_window_1m AS 58 | SELECT 59 | TUMBLE_START(log_time, INTERVAL '1' MINUTE) AS window_start, 60 | TUMBLE_ROWTIME(log_time, INTERVAL '1' MINUTE) AS window_end, 61 | SUM(size) AS total_size, 62 | COUNT(*) AS num_requests 63 | FROM server_logs 64 | GROUP BY 65 | TUMBLE(log_time, INTERVAL '1' MINUTE); 66 | 67 | 68 | CREATE TEMPORARY VIEW server_logs_window_5m AS 69 | SELECT 70 | TUMBLE_START(window_end, INTERVAL '5' MINUTE) AS window_start, 71 | TUMBLE_ROWTIME(window_end, INTERVAL '5' MINUTE) AS window_end, 72 | SUM(total_size) AS total_size, 73 | SUM(num_requests) AS num_requests 74 | FROM server_logs_window_1m 75 | GROUP BY 76 | TUMBLE(window_end, INTERVAL '5' MINUTE); 77 | 78 | BEGIN STATEMENT SET; 79 | 80 | INSERT INTO avg_request_size_1m SELECT 81 | window_start, 82 | window_end, 83 | total_size/num_requests AS avg_size 84 | FROM server_logs_window_1m; 85 | 86 | INSERT INTO avg_request_size_5m SELECT 87 | window_start, 88 | window_end, 89 | total_size/num_requests AS avg_size 90 | FROM server_logs_window_5m; 91 | 92 | END; 93 | ``` 94 | 95 | ## Example Output 96 | 97 | ### JobGraph 98 | 99 | ![jobgraph_chained](https://user-images.githubusercontent.com/23521087/105503848-5e94f600-5cc7-11eb-9a7f-2944dd4e1faf.png) 100 | 101 | ### Result 1 Minute Aggregations 102 | 103 | ![results_1m](https://user-images.githubusercontent.com/23521087/105503896-6c4a7b80-5cc7-11eb-958d-05d48c9921cf.png) 104 | 105 | ## Non-Chained Windows 106 | 107 |
108 | Non-Chained Windows 109 | 110 | ### Script 111 | 112 | ```shell script 113 | CREATE TEMPORARY TABLE server_logs ( 114 | log_time TIMESTAMP(3), 115 | client_ip STRING, 116 | client_identity STRING, 117 | userid STRING, 118 | request_line STRING, 119 | status_code STRING, 120 | size INT, 121 | WATERMARK FOR log_time AS log_time - INTERVAL '15' SECONDS 122 | ) WITH ( 123 | 'connector' = 'faker', 124 | 'fields.client_ip.expression' = '#{Internet.publicIpV4Address}', 125 | 'fields.client_identity.expression' = '-', 126 | 'fields.userid.expression' = '-', 127 | 'fields.log_time.expression' = '#{date.past ''15'',''5'',''SECONDS''}', 128 | 'fields.request_line.expression' = '#{regexify ''(GET|POST|PUT|PATCH){1}''} #{regexify ''(/search\.html|/login\.html|/prod\.html|cart\.html|/order\.html){1}''} #{regexify ''(HTTP/1\.1|HTTP/2|/HTTP/1\.0){1}''}', 129 | 'fields.status_code.expression' = '#{regexify ''(200|201|204|400|401|403|301){1}''}', 130 | 'fields.size.expression' = '#{number.numberBetween ''100'',''10000000''}' 131 | ); 132 | 133 | CREATE TEMPORARY TABLE avg_request_size_1m ( 134 | window_start TIMESTAMP(3), 135 | window_end TIMESTAMP(3), 136 | avg_size BIGINT 137 | ) 138 | WITH ( 139 | 'connector' = 'blackhole' 140 | ); 141 | 142 | CREATE TEMPORARY TABLE avg_request_size_5m ( 143 | window_start TIMESTAMP(3), 144 | window_end TIMESTAMP(3), 145 | avg_size BIGINT 146 | ) 147 | WITH ( 148 | 'connector' = 'blackhole' 149 | ); 150 | 151 | CREATE TEMPORARY VIEW server_logs_window_1m AS 152 | SELECT 153 | TUMBLE_START(log_time, INTERVAL '1' MINUTE) AS window_start, 154 | TUMBLE_ROWTIME(log_time, INTERVAL '1' MINUTE) AS window_end, 155 | SUM(size) AS total_size, 156 | COUNT(*) AS num_requests 157 | FROM server_logs 158 | GROUP BY 159 | TUMBLE(log_time, INTERVAL '1' MINUTE); 160 | 161 | 162 | CREATE TEMPORARY VIEW server_logs_window_5m AS 163 | SELECT 164 | TUMBLE_START(log_time, INTERVAL '5' MINUTE) AS window_start, 165 | TUMBLE_ROWTIME(log_time, INTERVAL '5' MINUTE) AS window_end, 166 | SUM(size) AS total_size, 167 | COUNT(*) AS num_requests 168 | FROM server_logs 169 | GROUP BY 170 | TUMBLE(log_time, INTERVAL '5' MINUTE); 171 | 172 | BEGIN STATEMENT SET; 173 | 174 | INSERT INTO avg_request_size_1m SELECT 175 | window_start, 176 | window_end, 177 | total_size/num_requests AS avg_size 178 | FROM server_logs_window_1m; 179 | 180 | INSERT INTO avg_request_size_5m SELECT 181 | window_start, 182 | window_end, 183 | total_size/num_requests AS avg_size 184 | FROM server_logs_window_5m; 185 | 186 | END; 187 | ``` 188 | 189 | ### Example Output 190 | 191 | #### JobGraph 192 | 193 | ![jobgraph_non_chained](https://user-images.githubusercontent.com/23521087/105503946-79676a80-5cc7-11eb-9e8e-15d39482fee9.png) 194 | 195 |
196 | -------------------------------------------------------------------------------- /joins/05_star_schema/05_star_schema.md: -------------------------------------------------------------------------------- 1 | # 05 Real Time Star Schema Denormalization (N-Way Join) 2 | 3 | > :bulb: In this recipe, we will de-normalize a simple star schema with an n-way temporal table join. 4 | 5 | [Star schemas](https://en.wikipedia.org/wiki/Star_schema) are a popular way of normalizing data within a data warehouse. 6 | At the center of a star schema is a **fact table** whose rows contain metrics, measurements, and other facts about the world. 7 | Surrounding fact tables are one or more **dimension tables** which have metadata useful for enriching facts when computing queries. 8 | You are running a small data warehouse for a railroad company which consists of a fact table (`train_activity`) and three dimension tables (`stations`, `booking_channels`, and `passengers`). 9 | All inserts to the fact table, and all updates to the dimension tables, are mirrored to Apache Kafka. 10 | Records in the fact table are interpreted as inserts only, and so the table is backed by the [standard Kafka connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/kafka.html) (`connector` = `kafka`);. 11 | In contrast, the records in the dimensional tables are upserts based on a primary key, which requires the [Upsert Kafka connector](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/connectors/upsert-kafka.html) (`connector` = `upsert-kafka`). 12 | 13 | With Flink SQL you can now easily join all dimensions to our fact table using a 5-way temporal table join. 14 | Temporal table joins take an arbitrary table (left input/probe site) and correlate each row to the corresponding row’s relevant version in a versioned table (right input/build side). 15 | Flink uses the SQL syntax of ``FOR SYSTEM_TIME AS OF`` to perform this operation. 16 | Using a temporal table join leads to consistent, reproducible results when joining a fact table with more (slowly) changing dimensional tables. 17 | Every event (row in the fact table) is joined to its corresponding value of each dimension based on when the event occurred in the real world. 18 | 19 | ## Script 20 | 21 | ```sql 22 | CREATE TEMPORARY TABLE passengers ( 23 | passenger_key STRING, 24 | first_name STRING, 25 | last_name STRING, 26 | update_time TIMESTAMP(3), 27 | WATERMARK FOR update_time AS update_time - INTERVAL '10' SECONDS, 28 | PRIMARY KEY (passenger_key) NOT ENFORCED 29 | ) WITH ( 30 | 'connector' = 'upsert-kafka', 31 | 'topic' = 'passengers', 32 | 'properties.bootstrap.servers' = 'localhost:9092', 33 | 'key.format' = 'raw', 34 | 'value.format' = 'json' 35 | ); 36 | 37 | CREATE TEMPORARY TABLE stations ( 38 | station_key STRING, 39 | update_time TIMESTAMP(3), 40 | city STRING, 41 | WATERMARK FOR update_time AS update_time - INTERVAL '10' SECONDS, 42 | PRIMARY KEY (station_key) NOT ENFORCED 43 | ) WITH ( 44 | 'connector' = 'upsert-kafka', 45 | 'topic' = 'stations', 46 | 'properties.bootstrap.servers' = 'localhost:9092', 47 | 'key.format' = 'raw', 48 | 'value.format' = 'json' 49 | ); 50 | 51 | CREATE TEMPORARY TABLE booking_channels ( 52 | booking_channel_key STRING, 53 | update_time TIMESTAMP(3), 54 | channel STRING, 55 | WATERMARK FOR update_time AS update_time - INTERVAL '10' SECONDS, 56 | PRIMARY KEY (booking_channel_key) NOT ENFORCED 57 | ) WITH ( 58 | 'connector' = 'upsert-kafka', 59 | 'topic' = 'booking_channels', 60 | 'properties.bootstrap.servers' = 'localhost:9092', 61 | 'key.format' = 'raw', 62 | 'value.format' = 'json' 63 | ); 64 | 65 | CREATE TEMPORARY TABLE train_activities ( 66 | scheduled_departure_time TIMESTAMP(3), 67 | actual_departure_date TIMESTAMP(3), 68 | passenger_key STRING, 69 | origin_station_key STRING, 70 | destination_station_key STRING, 71 | booking_channel_key STRING, 72 | WATERMARK FOR actual_departure_date AS actual_departure_date - INTERVAL '10' SECONDS 73 | ) WITH ( 74 | 'connector' = 'kafka', 75 | 'topic' = 'train_activities', 76 | 'properties.bootstrap.servers' = 'localhost:9092', 77 | 'value.format' = 'json', 78 | 'value.fields-include' = 'ALL' 79 | ); 80 | 81 | SELECT 82 | t.actual_departure_date, 83 | p.first_name, 84 | p.last_name, 85 | b.channel, 86 | os.city AS origin_station, 87 | ds.city AS destination_station 88 | FROM train_activities t 89 | LEFT JOIN booking_channels FOR SYSTEM_TIME AS OF t.actual_departure_date AS b 90 | ON t.booking_channel_key = b.booking_channel_key; 91 | LEFT JOIN passengers FOR SYSTEM_TIME AS OF t.actual_departure_date AS p 92 | ON t.passenger_key = p.passenger_key 93 | LEFT JOIN stations FOR SYSTEM_TIME AS OF t.actual_departure_date AS os 94 | ON t.origin_station_key = os.station_key 95 | LEFT JOIN stations FOR SYSTEM_TIME AS OF t.actual_departure_date AS ds 96 | ON t.destination_station_key = ds.station_key; 97 | ``` 98 | 99 | ## Example Output 100 | 101 | ### SQL Client 102 | 103 | ![05_output](https://user-images.githubusercontent.com/23521087/105504672-54272c00-5cc8-11eb-88da-901bb0006da1.png) 104 | 105 | ### JobGraph 106 | 107 | ![05_jobgraph](https://user-images.githubusercontent.com/23521087/105504615-440f4c80-5cc8-11eb-94f2-d07d0315dec5.png) 108 | 109 | ## Data Generators 110 | 111 |
112 | Data Generators 113 | 114 | The four topics are populated with Flink SQL jobs, too. 115 | We use the [`faker` connector](https://flink-packages.org/packages/flink-faker) to generate rows in memory based on Java Faker expressions and write those to the respective Kafka topics. 116 | 117 | ### ``train_activities`` Topic 118 | 119 | ### Script 120 | 121 | ```sql 122 | CREATE TEMPORARY TABLE train_activities_faker 123 | WITH ( 124 | 'connector' = 'faker', 125 | 'fields.scheduled_departure_time.expression' = '#{date.past ''10'',''0'',''SECONDS''}', 126 | 'fields.actual_departure_date.expression' = '#{date.past ''10'',''5'',''SECONDS''}', 127 | 'fields.passenger_key.expression' = '#{number.numberBetween ''0'',''10000000''}', 128 | 'fields.origin_station_key.expression' = '#{number.numberBetween ''0'',''1000''}', 129 | 'fields.destination_station_key.expression' = '#{number.numberBetween ''0'',''1000''}', 130 | 'fields.booking_channel_key.expression' = '#{number.numberBetween ''0'',''7''}', 131 | 'rows-per-second' = '1000' 132 | ) LIKE train_activities (EXCLUDING OPTIONS); 133 | 134 | INSERT INTO train_activities SELECT * FROM train_activities_faker; 135 | ``` 136 | #### Kafka Topic 137 | 138 | ```shell script 139 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic train_actitivies --property print.key=true --property key.separator=" - " 140 | null - {"scheduled_departure_time":"2020-12-19 13:52:37","actual_departure_date":"2020-12-19 13:52:16","passenger_key":7014937,"origin_station_key":577,"destination_station_key":862,"booking_channel_key":2} 141 | null - {"scheduled_departure_time":"2020-12-19 13:52:38","actual_departure_date":"2020-12-19 13:52:23","passenger_key":2244807,"origin_station_key":735,"destination_station_key":739,"booking_channel_key":2} 142 | null - {"scheduled_departure_time":"2020-12-19 13:52:46","actual_departure_date":"2020-12-19 13:52:18","passenger_key":2605313,"origin_station_key":216,"destination_station_key":453,"booking_channel_key":3} 143 | null - {"scheduled_departure_time":"2020-12-19 13:53:13","actual_departure_date":"2020-12-19 13:52:19","passenger_key":7111654,"origin_station_key":234,"destination_station_key":833,"booking_channel_key":5} 144 | null - {"scheduled_departure_time":"2020-12-19 13:52:22","actual_departure_date":"2020-12-19 13:52:17","passenger_key":2847474,"origin_station_key":763,"destination_station_key":206,"booking_channel_key":3} 145 | ``` 146 | 147 | ### ``passengers`` Topic 148 | 149 | #### Script 150 | 151 | ```sql 152 | CREATE TEMPORARY TABLE passengers_faker 153 | WITH ( 154 | 'connector' = 'faker', 155 | 'fields.passenger_key.expression' = '#{number.numberBetween ''0'',''10000000''}', 156 | 'fields.update_time.expression' = '#{date.past ''10'',''5'',''SECONDS''}', 157 | 'fields.first_name.expression' = '#{Name.firstName}', 158 | 'fields.last_name.expression' = '#{Name.lastName}', 159 | 'rows-per-second' = '1000' 160 | ) LIKE passengers (EXCLUDING OPTIONS); 161 | 162 | INSERT INTO passengers SELECT * FROM passengers_faker; 163 | ``` 164 | 165 | #### Kafka Topic 166 | 167 | ```shell script 168 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic passengers --property print.key=true --property key.separator=" - " 169 | 749049 - {"passenger_key":"749049","first_name":"Booker","last_name":"Hackett","update_time":"2020-12-19 14:02:32"} 170 | 7065702 - {"passenger_key":"7065702","first_name":"Jeramy","last_name":"Breitenberg","update_time":"2020-12-19 14:02:38"} 171 | 3690329 - {"passenger_key":"3690329","first_name":"Quiana","last_name":"Macejkovic","update_time":"2020-12-19 14:02:27"} 172 | 1212728 - {"passenger_key":"1212728","first_name":"Lawerence","last_name":"Simonis","update_time":"2020-12-19 14:02:27"} 173 | 6993699 - {"passenger_key":"6993699","first_name":"Ardelle","last_name":"Frami","update_time":"2020-12-19 14:02:19"} 174 | ``` 175 | 176 | ### ``stations`` Topic 177 | 178 | #### Script 179 | 180 | ```sql 181 | CREATE TEMPORARY TABLE stations_faker 182 | WITH ( 183 | 'connector' = 'faker', 184 | 'fields.station_key.expression' = '#{number.numberBetween ''0'',''1000''}', 185 | 'fields.city.expression' = '#{Address.city}', 186 | 'fields.update_time.expression' = '#{date.past ''10'',''5'',''SECONDS''}', 187 | 'rows-per-second' = '100' 188 | ) LIKE stations (EXCLUDING OPTIONS); 189 | 190 | INSERT INTO stations SELECT * FROM stations_faker; 191 | ``` 192 | 193 | #### Kafka Topic 194 | 195 | ```shell script 196 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic stations --property print.key=true --property key.separator=" - " 197 | 80 - {"station_key":"80","update_time":"2020-12-19 13:59:20","city":"Harlandport"} 198 | 33 - {"station_key":"33","update_time":"2020-12-19 13:59:12","city":"North Georgine"} 199 | 369 - {"station_key":"369","update_time":"2020-12-19 13:59:12","city":"Tillmanhaven"} 200 | 580 - {"station_key":"580","update_time":"2020-12-19 13:59:12","city":"West Marianabury"} 201 | 616 - {"station_key":"616","update_time":"2020-12-19 13:59:09","city":"West Sandytown"} 202 | ``` 203 | 204 | ### ``booking_channels`` Topic 205 | 206 | #### Script 207 | 208 | ```sql 209 | CREATE TEMPORARY TABLE booking_channels_faker 210 | WITH ( 211 | 'connector' = 'faker', 212 | 'fields.booking_channel_key.expression' = '#{number.numberBetween ''0'',''7''}', 213 | 'fields.channel.expression' = '#{regexify ''(bahn\.de|station|retailer|app|lidl|hotline|joyn){1}''}', 214 | 'fields.update_time.expression' = '#{date.past ''10'',''5'',''SECONDS''}', 215 | 'rows-per-second' = '100' 216 | ) LIKE booking_channels (EXCLUDING OPTIONS); 217 | 218 | INSERT INTO booking_channels SELECT * FROM booking_channels_faker; 219 | ``` 220 | 221 | #### Kafka Topic 222 | 223 | ```shell script 224 | ➜ bin ./kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic booking_channels --property print.key=true --property key.separator=" - " 225 | 1 - {"booking_channel_key":"1","update_time":"2020-12-19 13:57:05","channel":"joyn"} 226 | 0 - {"booking_channel_key":"0","update_time":"2020-12-19 13:57:17","channel":"station"} 227 | 4 - {"booking_channel_key":"4","update_time":"2020-12-19 13:57:15","channel":"joyn"} 228 | 2 - {"booking_channel_key":"2","update_time":"2020-12-19 13:57:02","channel":"app"} 229 | 1 - {"booking_channel_key":"1","update_time":"2020-12-19 13:57:06","channel":"retailer"} 230 | 231 | ``` 232 | 233 |
234 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright 2020 Ververica GmbH 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. -------------------------------------------------------------------------------- /other-builtin-functions/04_override_table_options/airports.csv: -------------------------------------------------------------------------------- 1 | ABE,Lehigh Valley International Airport,Allentown,PA,USA,40.65236,-75.44040 2 | ABI,Abilene Regional Airport,Abilene,TX,USA,32.41132,-99.68190 3 | ABQ,Albuquerque International Sunport,Albuquerque,NM,USA,35.04022,-106.60919 4 | ABR,Aberdeen Regional Airport,Aberdeen,SD,USA,45.44906,-98.42183 5 | ABY,Southwest Georgia Regional Airport,Albany,GA,USA,31.53552,-84.19447 6 | ACK,Nantucket Memorial Airport,Nantucket,MA,USA,41.25305,-70.06018 7 | ACT,Waco Regional Airport,Waco,TX,USA,31.61129,-97.23052 8 | ACV,Arcata Airport,Arcata/Eureka,CA,USA,40.97812,-124.10862 9 | ACY,Atlantic City International Airport,Atlantic City,NJ,USA,39.45758,-74.57717 10 | ADK,Adak Airport,Adak,AK,USA,51.87796,-176.64603 11 | ADQ,Kodiak Airport,Kodiak,AK,USA,57.74997,-152.49386 12 | AEX,Alexandria International Airport,Alexandria,LA,USA,31.32737,-92.54856 13 | AGS,Augusta Regional Airport (Bush Field),Augusta,GA,USA,33.36996,-81.96450 14 | AKN,King Salmon Airport,King Salmon,AK,USA,58.67680,-156.64922 15 | ALB,Albany International Airport,Albany,NY,USA,42.74812,-73.80298 16 | ALO,Waterloo Regional Airport,Waterloo,IA,USA,42.55708,-92.40034 17 | AMA,Rick Husband Amarillo International Airport,Amarillo,TX,USA,35.21937,-101.70593 18 | ANC,Ted Stevens Anchorage International Airport,Anchorage,AK,USA,61.17432,-149.99619 19 | APN,Alpena County Regional Airport,Alpena,MI,USA,45.07807,-83.56029 20 | ASE,Aspen-Pitkin County Airport,Aspen,CO,USA,39.22316,-106.86885 21 | ATL,Hartsfield-Jackson Atlanta International Airport,Atlanta,GA,USA,33.64044,-84.42694 22 | ATW,Appleton International Airport,Appleton,WI,USA,44.25741,-88.51948 23 | AUS,Austin-Bergstrom International Airport,Austin,TX,USA,30.19453,-97.66987 24 | AVL,Asheville Regional Airport,Asheville,NC,USA,35.43619,-82.54181 25 | AVP,Wilkes-Barre/Scranton International Airport,Wilkes-Barre/Scranton,PA,USA,41.33815,-75.72427 26 | AZO,Kalamazoo/Battle Creek International Airport,Kalamazoo,MI,USA,42.23488,-85.55206 27 | BDL,Bradley International Airport,Windsor Locks,CT,USA,41.93887,-72.68323 28 | BET,Bethel Airport,Bethel,AK,USA,60.77978,-161.83800 29 | BFL,Meadows Field,Bakersfield,CA,USA,35.43360,-119.05677 30 | BGM,Greater Binghamton Airport,Binghamton,NY,USA,42.20848,-75.97961 31 | BGR,Bangor International Airport,Bangor,ME,USA,44.80744,-68.82814 32 | BHM,Birmingham-Shuttlesworth International Airport,Birmingham,AL,USA,33.56294,-86.75355 33 | BIL,Billings Logan International Airport,Billings,MT,USA,45.80766,-108.54286 34 | BIS,Bismarck Municipal Airport,Bismarck,ND,USA,46.77411,-100.74672 35 | BJI,Bemidji Regional Airport,Bemidji,MN,USA,47.50942,-94.93372 36 | BLI,Bellingham International Airport,Bellingham,WA,USA,48.79275,-122.53753 37 | BMI,Central Illinois Regional Airport at Bloomington-Normal,Bloomington,IL,USA,40.47799,-88.91595 38 | BNA,Nashville International Airport,Nashville,TN,USA,36.12448,-86.67818 39 | BOI,Boise Airport (Boise Air Terminal),Boise,ID,USA,43.56444,-116.22278 40 | BOS,Gen. Edward Lawrence Logan International Airport,Boston,MA,USA,42.36435,-71.00518 41 | BPT,Jack Brooks Regional Airport (Southeast Texas Regional),Beaumont/Port Arthur,TX,USA,29.95083,-94.02069 42 | BQK,Brunswick Golden Isles Airport,Brunswick,GA,USA,31.25903,-81.46631 43 | BQN,Rafael Hernández Airport,Aguadilla,PR,USA,18.49486,-67.12944 44 | BRD,Brainerd Lakes Regional Airport,Brainerd,MN,USA,46.39786,-94.13723 45 | BRO,Brownsville/South Padre Island International Airport,Brownsville,TX,USA,25.90683,-97.42586 46 | BRW,Wiley Post-Will Rogers Memorial Airport,Barrow,AK,USA,71.28545,-156.76600 47 | BTM,Bert Mooney Airport,Butte,MT,USA,45.95480,-112.49746 48 | BTR,Baton Rouge Metropolitan Airport,Baton Rouge,LA,USA,30.53316,-91.14963 49 | BTV,Burlington International Airport,Burlington,VT,USA,44.47300,-73.15031 50 | BUF,Buffalo Niagara International Airport,Buffalo,NY,USA,42.94052,-78.73217 51 | BUR,Bob Hope Airport (Hollywood Burbank Airport),Burbank,CA,USA,34.20062,-118.35850 52 | BWI,Baltimore-Washington International Airport,Baltimore,MD,USA,39.17540,-76.66820 53 | BZN,Bozeman Yellowstone International Airport (Gallatin Field Airport),Bozeman,MT,USA,45.77690,-111.15301 54 | CAE,Columbia Metropolitan Airport,Columbia,SC,USA,33.93884,-81.11954 55 | CAK,Akron-Canton Regional Airport,Akron,OH,USA,40.91631,-81.44247 56 | CDC,Cedar City Regional Airport,Cedar City,UT,USA,37.70097,-113.09858 57 | CDV,Merle K. (Mudhole) Smith Airport,Cordova,AK,USA,60.49183,-145.47765 58 | CEC,Del Norte County Airport (Jack McNamara Field),Crescent City,CA,USA,41.78016,-124.23653 59 | CHA,Chattanooga Metropolitan Airport (Lovell Field),Chattanooga,TN,USA,35.03527,-85.20379 60 | CHO,Charlottesville-Albemarle Airport,Charlottesville,VA,USA,38.13864,-78.45286 61 | CHS,Charleston International Airport/Charleston AFB,Charleston,SC,USA,32.89865,-80.04051 62 | CID,The Eastern Iowa Airport,Cedar Rapids,IA,USA,41.88459,-91.71087 63 | CIU,Chippewa County International Airport,Sault Ste. Marie,MI,USA,46.25075,-84.47239 64 | CLD,McClellan-Palomar Airport,San Diego,CA,USA,33.12723,-117.27873 65 | CLE,Cleveland Hopkins International Airport,Cleveland,OH,USA,41.41089,-81.84940 66 | CLL,Easterwood Airport,College Station,TX,USA,30.58859,-96.36382 67 | CLT,Charlotte Douglas International Airport,Charlotte,NC,USA,35.21401,-80.94313 68 | CMH,Port Columbus International Airport,Columbus,OH,USA,39.99799,-82.89188 69 | CMI,University of Illinois - Willard Airport,Champaign/Urbana,IL,USA,40.03925,-88.27806 70 | CMX,Houghton County Memorial Airport,Hancock,MI,USA,47.16842,-88.48906 71 | CNY,Canyonlands Field,Moab,UT,USA,38.75496,-109.75484 72 | COD,Yellowstone Regional Airport,Cody,WY,USA,44.52019,-109.02380 73 | COS,City of Colorado Springs Municipal Airport,Colorado Springs,CO,USA,38.80581,-104.70025 74 | COU,Columbia Regional Airport,Columbia,MO,USA,38.81809,-92.21963 75 | CPR,Natrona County International Airport,Casper,WY,USA,42.90836,-106.46447 76 | CRP,Corpus Christi International Airport,Corpus Christi,TX,USA,27.77036,-97.50122 77 | CRW,Yeager Airport,Charleston,WV,USA,38.37315,-81.59319 78 | CSG,Columbus Metropolitan Airport,Columbus,GA,USA,32.51633,-84.93886 79 | CVG,Cincinnati/Northern Kentucky International Airport,Covington,KY,USA,39.04614,-84.66217 80 | CWA,Central Wisconsin Airport,Mosinee,WI,USA,44.77762,-89.66678 81 | DAB,Daytona Beach International Airport,Daytona Beach,FL,USA,29.17992,-81.05806 82 | DAL,Dallas Love Field,Dallas,TX,USA,32.84711,-96.85177 83 | DAY,James M. Cox Dayton International Airport,Dayton,OH,USA,39.90238,-84.21938 84 | DBQ,Dubuque Regional Airport,Dubuque,IA,USA,42.40296,-90.70917 85 | DCA,Ronald Reagan Washington National Airport,Arlington,VA,USA,38.85208,-77.03772 86 | DEN,Denver International Airport,Denver,CO,USA,39.85841,-104.66700 87 | DFW,Dallas/Fort Worth International Airport,Dallas-Fort Worth,TX,USA,32.89595,-97.03720 88 | DHN,Dothan Regional Airport,Dothan,AL,USA,31.32134,-85.44963 89 | DIK,Dickinson Theodore Roosevelt Regional Airport,Dickinson,ND,USA,46.79739,-102.80195 90 | DLG,Dillingham Airport,Dillingham,AK,USA,59.04541,-158.50334 91 | DLH,Duluth International Airport,Duluth,MN,USA,46.84209,-92.19365 92 | DRO,Durango-La Plata County Airport,Durango,CO,USA,37.15152,-107.75377 93 | DSM,Des Moines International Airport,Des Moines,IA,USA,41.53493,-93.66068 94 | DTW,Detroit Metropolitan Airport,Detroit,MI,USA,42.21206,-83.34884 95 | DVL,Devils Lake Regional Airport,Devils Lake,ND,USA,48.11425,-98.90878 96 | EAU,Chippewa Valley Regional Airport,Eau Claire,WI,USA,44.86526,-91.48507 97 | ECP,Northwest Florida Beaches International Airport,Panama City,FL,USA,, 98 | EGE,Eagle County Regional Airport,Eagle,CO,USA,39.64257,-106.91770 99 | EKO,Elko Regional Airport,Elko,NV,USA,40.82493,-115.79170 100 | ELM,Elmira/Corning Regional Airport,Elmira,NY,USA,42.15991,-76.89144 101 | ELP,El Paso International Airport,El Paso,TX,USA,31.80667,-106.37781 102 | ERI,Erie International Airport,Erie,PA,USA,42.08202,-80.17622 103 | ESC,Delta County Airport,Escanaba,MI,USA,45.72267,-87.09373 104 | EUG,Eugene Airport (Mahlon Sweet Field),Eugene,OR,USA,44.12326,-123.21869 105 | EVV,Evansville Regional Airport,Evansville,IN,USA,38.03799,-87.53063 106 | EWN,Coastal Carolina Regional Airport (Craven County Regional),New Bern,NC,USA,35.07297,-77.04294 107 | EWR,Newark Liberty International Airport,Newark,NJ,USA,40.69250,-74.16866 108 | EYW,Key West International Airport,Key West,FL,USA,24.55611,-81.75956 109 | FAI,Fairbanks International Airport,Fairbanks,AK,USA,64.81368,-147.85967 110 | FAR,Hector International Airport,Fargo,ND,USA,46.91935,-96.81499 111 | FAT,Fresno Yosemite International Airport,Fresno,CA,USA,36.77619,-119.71814 112 | FAY,Fayetteville Regional Airport,Fayetteville,NC,USA,34.99147,-78.88000 113 | FCA,Glacier Park International Airport,Kalispell,MT,USA,48.31140,-114.25507 114 | FLG,Flagstaff Pulliam Airport,Flagstaff,AZ,USA,35.13845,-111.67122 115 | FLL,Fort Lauderdale-Hollywood International Airport,Ft. Lauderdale,FL,USA,26.07258,-80.15275 116 | FNT,Bishop International Airport,Flint,MI,USA,42.96550,-83.74346 117 | FSD,Sioux Falls Regional Airport,Sioux Falls,SD,USA,43.58135,-96.74170 118 | FSM,Fort Smith Regional Airport,Fort Smith,AR,USA,35.33659,-94.36744 119 | FWA,Fort Wayne International Airport,Fort Wayne,IN,USA,40.97847,-85.19515 120 | GCC,Gillette-Campbell County Airport,Gillette,WY,USA,44.34890,-105.53936 121 | GCK,Garden City Regional Airport,Garden City,KS,USA,37.92752,-100.72441 122 | GEG,Spokane International Airport,Spokane,WA,USA,47.61986,-117.53384 123 | GFK,Grand Forks International Airport,Grand Forks,ND,USA,47.94926,-97.17611 124 | GGG,East Texas Regional Airport,Longview,TX,USA,32.38486,-94.71171 125 | GJT,Grand Junction Regional Airport (Walker Field),Grand Junction,CO,USA,39.12241,-108.52673 126 | GNV,Gainesville Regional Airport,Gainesville,FL,USA,29.69006,-82.27178 127 | GPT,Gulfport-Biloxi International Airport,Gulfport-Biloxi,MS,USA,30.40728,-89.07009 128 | GRB,Green Bay-Austin Straubel International Airport,Green Bay,WI,USA,44.48507,-88.12959 129 | GRI,Central Nebraska Regional Airport,Grand Island,NE,USA,40.96747,-98.30861 130 | GRK,Killeen-Fort Hood Regional Airport,Killeen,TX,USA,31.06490,-97.82780 131 | GRR,Gerald R. Ford International Airport,Grand Rapids,MI,USA,42.88082,-85.52277 132 | GSO,Piedmont Triad International Airport,Greensboro,NC,USA,36.09775,-79.93730 133 | GSP,Greenville-Spartanburg International Airport,Greer,SC,USA,34.89567,-82.21886 134 | GST,Gustavus Airport,Gustavus,AK,USA,58.42438,-135.70738 135 | GTF,Great Falls International Airport,Great Falls,MT,USA,47.48200,-111.37069 136 | GTR,Golden Triangle Regional Airport,Columbus-Starkville-West Point,MS,USA,33.45033,-88.59137 137 | GUC,Gunnison-Crested Butte Regional Airport,Gunnison,CO,USA,38.53396,-106.93318 138 | GUM,Guam International Airport,Agana,GU,USA,13.48345,-144.79598 139 | HDN,Yampa Valley Airport (Yampa Valley Regional),Hayden,CO,USA,40.48118,-107.21766 140 | HIB,Range Regional Airport (Chisholm-Hibbing Airport),Hibbing,MN,USA,47.38660,-92.83899 141 | HLN,Helena Regional Airport,Helena,MT,USA,46.60682,-111.98275 142 | HNL,Honolulu International Airport,Honolulu,HI,USA,21.31869,-157.92241 143 | HOB,Lea County Regional Airport,Hobbs,NM,USA,32.68753,-103.21703 144 | HOU,William P. Hobby Airport,Houston,TX,USA,29.64542,-95.27889 145 | HPN,Westchester County Airport,White Plains,NY,USA,41.06696,-73.70757 146 | HRL,Valley International Airport,Harlingen,TX,USA,26.22851,-97.65439 147 | HSV,Huntsville International Airport,Huntsville,AL,USA,34.64045,-86.77311 148 | HYA,Barnstable Municipal Airport,Hyannis,MA,USA,41.66934,-70.28036 149 | HYS,Hays Regional Airport,Hays,KS,USA,38.84494,-99.27403 150 | IAD,Washington Dulles International Airport,Chantilly,VA,USA,38.94453,-77.45581 151 | IAG,Niagara Falls International Airport,Niagara Falls,NY,USA,43.10726,-78.94538 152 | IAH,George Bush Intercontinental Airport,Houston,TX,USA,29.98047,-95.33972 153 | ICT,Wichita Dwight D. Eisenhower National Airport (Wichita Mid-Continent Airport),Wichita,KS,USA,37.64996,-97.43305 154 | IDA,Idaho Falls Regional Airport,Idaho Falls,ID,USA,43.51456,-112.07017 155 | ILG,Wilmington Airport,Wilmington,DE,USA,39.67872,-75.60653 156 | ILM,Wilmington International Airport,Wilmington,NC,USA,34.27061,-77.90256 157 | IMT,Ford Airport,Iron Mountain/Kingsford,MI,USA,45.81835,-88.11454 158 | IND,Indianapolis International Airport,Indianapolis,IN,USA,39.71733,-86.29438 159 | INL,Falls International Airport,International Falls,MN,USA,48.56619,-93.40307 160 | ISN,Sloulin Field International Airport,Williston,ND,USA,48.17794,-103.64235 161 | ISP,Long Island MacArthur Airport,Islip,NY,USA,40.79524,-73.10021 162 | ITH,Ithaca Tompkins Regional Airport,Ithaca,NY,USA,42.49103,-76.45844 163 | ITO,Hilo International Airport,Hilo,HI,USA,19.72026,-155.04847 164 | JAC,Jackson Hole Airport,Jackson,WY,USA,43.60732,-110.73774 165 | JAN,Jackson-Evers International Airport,Jackson,MS,USA,32.31117,-90.07589 166 | JAX,Jacksonville International Airport,Jacksonville,FL,USA,30.49406,-81.68786 167 | JFK,John F. Kennedy International Airport (New York International Airport),New York,NY,USA,40.63975,-73.77893 168 | JLN,Joplin Regional Airport,Joplin,MO,USA,37.15181,-94.49827 169 | JMS,Jamestown Regional Airport,Jamestown,ND,USA,46.92972,-98.67820 170 | JNU,Juneau International Airport,Juneau,AK,USA,58.35496,-134.57628 171 | KOA,Kona International Airport at Keahole,Kailua/Kona,HI,USA,19.73877,-156.04563 172 | KTN,Ketchikan International Airport,Ketchikan,AK,USA,55.35557,-131.71374 173 | LAN,Capital Region International Airport ( Lansing Capital City),Lansing,MI,USA,42.77870,-84.58736 174 | LAR,Laramie Regional Airport,Laramie,WY,USA,41.31205,-105.67499 175 | LAS,McCarran International Airport,Las Vegas,NV,USA,36.08036,-115.15233 176 | LAW,Lawton-Fort Sill Regional Airport,Lawton,OK,USA,34.56771,-98.41664 177 | LAX,Los Angeles International Airport,Los Angeles,CA,USA,33.94254,-118.40807 178 | LBB,Lubbock Preston Smith International Airport,Lubbock,TX,USA,33.66364,-101.82278 179 | LBE,Arnold Palmer Regional Airport,Latrobe,PA,USA,40.27594,-79.40480 180 | LCH,Lake Charles Regional Airport,Lake Charles,LA,USA,30.12610,-93.22340 181 | LEX,Blue Grass Airport,Lexington,KY,USA,38.03697,-84.60539 182 | LFT,Lafayette Regional Airport,Lafayette,LA,USA,30.20528,-91.98766 183 | LGA,LaGuardia Airport (Marine Air Terminal),New York,NY,USA,40.77724,-73.87261 184 | LGB,Long Beach Airport (Daugherty Field),Long Beach,CA,USA,33.81772,-118.15161 185 | LIH,Lihue Airport,Lihue,HI,USA,21.97598,-159.33896 186 | LIT,Bill and Hillary Clinton National Airport (Adams Field),Little Rock,AR,USA,34.72940,-92.22425 187 | LNK,Lincoln Airport (Lincoln Municipal),Lincoln,NE,USA,40.85097,-96.75925 188 | LRD,Laredo International Airport,Laredo,TX,USA,27.54374,-99.46154 189 | LSE,La Crosse Regional Airport,La Crosse,WI,USA,43.87938,-91.25654 190 | LWS,Lewiston-Nez Perce County Airport,Lewiston,ID,USA,46.37450,-117.01539 191 | MAF,Midland International Airport,Midland,TX,USA,31.94253,-102.20191 192 | MBS,MBS International Airport,Saginaw,MI,USA,43.53291,-84.07965 193 | MCI,Kansas City International Airport,Kansas City,MO,USA,39.29761,-94.71391 194 | MCO,Orlando International Airport,Orlando,FL,USA,28.42889,-81.31603 195 | MDT,Harrisburg International Airport,Harrisburg,PA,USA,40.19350,-76.76340 196 | MDW,Chicago Midway International Airport,Chicago,IL,USA,41.78598,-87.75242 197 | MEI,Meridian Regional Airport,Meridian,MS,USA,32.33313,-88.75121 198 | MEM,Memphis International Airport,Memphis,TN,USA,35.04242,-89.97667 199 | MFE,McAllen-Miller International Airport (McAllen Miller International),McAllen,TX,USA,26.17583,-98.23861 200 | MFR,Rogue Valley International Airport,Medford,OR,USA,42.37423,-122.87350 201 | MGM,Montgomery Regional Airport,Montgomery,AL,USA,32.30064,-86.39398 202 | MHK,Manhattan Regional Airport,Manhattan,KS,USA,39.14097,-96.67083 203 | MHT,Manchester-Boston Regional Airport,Manchester,NH,USA,42.93452,-71.43706 204 | MIA,Miami International Airport,Miami,FL,USA,25.79325,-80.29056 205 | MKE,General Mitchell International Airport,Milwaukee,WI,USA,42.94722,-87.89658 206 | MKG,Muskegon County Airport,Muskegon,MI,USA,43.16949,-86.23822 207 | MLB,Melbourne International Airport,Melbourne,FL,USA,28.10275,-80.64581 208 | MLI,Quad City International Airport,Moline,IL,USA,41.44853,-90.50754 209 | MLU,Monroe Regional Airport,Monroe,LA,USA,32.51087,-92.03769 210 | MMH,Mammoth Yosemite Airport,Mammoth Lakes,CA,USA,37.62405,-118.83777 211 | MOB,Mobile Regional Airport,Mobile,AL,USA,30.69142,-88.24283 212 | MOT,Minot International Airport,Minot,ND,USA,48.25938,-101.28033 213 | MQT,Sawyer International Airport,Marquette,MI,USA,46.35364,-87.39536 214 | MRY,Monterey Regional Airport (Monterey Peninsula Airport),Monterey,CA,USA,36.58698,-121.84295 215 | MSN,Dane County Regional Airport,Madison,WI,USA,43.13986,-89.33751 216 | MSO,Missoula International Airport,Missoula,MT,USA,46.91631,-114.09056 217 | MSP,Minneapolis-Saint Paul International Airport,Minneapolis,MN,USA,44.88055,-93.21692 218 | MSY,Louis Armstrong New Orleans International Airport,New Orleans,LA,USA,29.99339,-90.25803 219 | MTJ,Montrose Regional Airport,Montrose,CO,USA,38.50887,-107.89383 220 | MVY,Martha's Vineyard Airport,Marthas Vineyard,MA,USA,41.39303,-70.61433 221 | MYR,Myrtle Beach International Airport,Myrtle Beach,SC,USA,33.67975,-78.92833 222 | OAJ,Albert J. Ellis Airport,Jacksonville,NC,USA,34.82916,-77.61214 223 | OAK,Oakland International Airport,Oakland,CA,USA,37.72129,-122.22072 224 | OGG,Kahului Airport,Kahului,HI,USA,20.89865,-156.43046 225 | OKC,Will Rogers World Airport,Oklahoma City,OK,USA,35.39309,-97.60073 226 | OMA,Eppley Airfield,Omaha,NE,USA,41.30252,-95.89417 227 | OME,Nome Airport,Nome,AK,USA,64.51220,-165.44525 228 | ONT,Ontario International Airport,Ontario,CA,USA,34.05600,-117.60119 229 | ORD,Chicago O'Hare International Airport,Chicago,IL,USA,41.97960,-87.90446 230 | ORF,Norfolk International Airport,Norfolk,VA,USA,36.89461,-76.20122 231 | ORH,Worcester Regional Airport,Worcester,MA,USA,42.26734,-71.87571 232 | OTH,Southwest Oregon Regional Airport (North Bend Municipal),North Bend,OR,USA,43.41714,-124.24603 233 | OTZ,Ralph Wien Memorial Airport,Kotzebue,AK,USA,66.88468,-162.59855 234 | PAH,Barkley Regional Airport,Paducah,KY,USA,37.06083,-88.77375 235 | PBG,Plattsburgh International Airport,Plattsburgh,NY,USA,, 236 | PBI,Palm Beach International Airport,West Palm Beach,FL,USA,26.68316,-80.09559 237 | PDX,Portland International Airport,Portland,OR,USA,45.58872,-122.59750 238 | PHF,Newport News/Williamsburg International Airport,Newport News,VA,USA,37.13190,-76.49299 239 | PHL,Philadelphia International Airport,Philadelphia,PA,USA,39.87195,-75.24114 240 | PHX,Phoenix Sky Harbor International Airport,Phoenix,AZ,USA,33.43417,-112.00806 241 | PIA,General Wayne A. Downing Peoria International Airport,Peoria,IL,USA,40.66424,-89.69331 242 | PIB,Hattiesburg-Laurel Regional Airport,Hattiesburg-Laurel,MS,USA,31.46715,-89.33706 243 | PIH,Pocatello Regional Airport,Pocatello,ID,USA,42.91131,-112.59586 244 | PIT,Pittsburgh International Airport,Pittsburgh,PA,USA,40.49147,-80.23287 245 | PLN,Pellston Regional Airport of Emmet County,Pellston,MI,USA,45.57093,-84.79672 246 | PNS,Pensacola International Airport (Pensacola Gulf Coast Regional Airport),Pensacola,FL,USA,30.47331,-87.18744 247 | PPG,Pago Pago International Airport (Tafuna Airport),Pago Pago,AS,USA,14.33102,-170.71053 248 | PSC,Tri-Cities Airport,Pasco,WA,USA,46.26468,-119.11903 249 | PSE,Mercedita Airport,Ponce,PR,USA,18.00830,-66.56301 250 | PSG,Petersburg James A. Johnson Airport,Petersburg,AK,USA,56.80165,-132.94528 251 | PSP,Palm Springs International Airport,Palm Springs,CA,USA,33.82922,-116.50625 252 | PUB,Pueblo Memorial Airport,Pueblo,CO,USA,38.28909,-104.49657 253 | PVD,Theodore Francis Green State Airport,Providence,RI,USA,41.72400,-71.42822 254 | PWM,Portland International Jetport,Portland,ME,USA,43.64617,-70.30875 255 | RAP,Rapid City Regional Airport,Rapid City,SD,USA,44.04532,-103.05737 256 | RDD,Redding Municipal Airport,Redding,CA,USA,40.50898,-122.29340 257 | RDM,Redmond Municipal Airport (Roberts Field),Redmond,OR,USA,44.25407,-121.14996 258 | RDU,Raleigh-Durham International Airport,Raleigh,NC,USA,35.87764,-78.78747 259 | RHI,Rhinelander-Oneida County Airport,Rhinelander,WI,USA,45.63119,-89.46745 260 | RIC,Richmond International Airport,Richmond,VA,USA,37.50517,-77.31967 261 | RKS,Rock Springs-Sweetwater County Airport,Rock Springs,WY,USA,41.59422,-109.06519 262 | RNO,Reno/Tahoe International Airport,Reno,NV,USA,39.49858,-119.76806 263 | ROA,Roanoke Regional Airport (Woodrum Field),Roanoke,VA,USA,37.32547,-79.97543 264 | ROC,Greater Rochester International Airport,Rochester,NY,USA,43.11887,-77.67238 265 | ROW,Roswell International Air Center,Roswell,NM,USA,33.30156,-104.53056 266 | RST,Rochester International Airport,Rochester,MN,USA,43.90883,-92.49799 267 | RSW,Southwest Florida International Airport,Ft. Myers,FL,USA,26.53617,-81.75517 268 | SAF,Santa Fe Municipal Airport,Santa Fe,NM,USA,35.61678,-106.08814 269 | SAN,San Diego International Airport (Lindbergh Field),San Diego,CA,USA,32.73356,-117.18966 270 | SAT,San Antonio International Airport,San Antonio,TX,USA,29.53369,-98.46978 271 | SAV,Savannah/Hilton Head International Airport,Savannah,GA,USA,32.12758,-81.20214 272 | SBA,Santa Barbara Municipal Airport (Santa Barbara Airport),Santa Barbara,CA,USA,34.42621,-119.84037 273 | SBN,South Bend International Airport (South Bend Regional),South Bend,IN,USA,41.70895,-86.31847 274 | SBP,San Luis Obispo County Regional Airport (McChesney Field),San Luis Obispo,CA,USA,35.23706,-120.64239 275 | SCC,Deadhorse Airport (Prudhoe Bay Airport),Deadhorse,AK,USA,70.19476,-148.46516 276 | SCE,University Park Airport,State College,PA,USA,40.85121,-77.84630 277 | SDF,Louisville International Airport (Standiford Field),Louisville,KY,USA,38.17439,-85.73600 278 | SEA,Seattle-Tacoma International Airport,Seattle,WA,USA,47.44898,-122.30931 279 | SFO,San Francisco International Airport,San Francisco,CA,USA,37.61900,-122.37484 280 | SGF,Springfield-Branson National Airport,Springfield,MO,USA,37.24433,-93.38686 281 | SGU,St. George Regional Airport,St George,UT,USA,37.09058,-113.59306 282 | SHV,Shreveport Regional Airport,Shreveport,LA,USA,32.44663,-93.82560 283 | SIT,Sitka Rocky Gutierrez Airport,Sitka,AK,USA,57.04714,-135.36160 284 | SJC,Norman Y. Mineta San José International Airport,San Jose,CA,USA,37.36186,-121.92901 285 | SJT,San Angelo Regional Airport (Mathis Field),San Angelo,TX,USA,31.35775,-100.49631 286 | SJU,Luis Muñoz Marín International Airport,San Juan,PR,USA,18.43942,-66.00183 287 | SLC,Salt Lake City International Airport,Salt Lake City,UT,USA,40.78839,-111.97777 288 | SMF,Sacramento International Airport,Sacramento,CA,USA,38.69542,-121.59077 289 | SMX,Santa Maria Public Airport (Capt G. Allan Hancock Field),Santa Maria,CA,USA,34.89925,-120.45758 290 | SNA,John Wayne Airport (Orange County Airport),Santa Ana,CA,USA,33.67566,-117.86822 291 | SPI,Abraham Lincoln Capital Airport,Springfield,IL,USA,39.84395,-89.67762 292 | SPS,Wichita Falls Municipal Airport/Sheppard AFB,Wichita Falls,TX,USA,33.98880,-98.49189 293 | SRQ,Sarasota-Bradenton International Airport,Sarasota,FL,USA,27.39533,-82.55411 294 | STC,St. Cloud Regional Airport,St Cloud,MN,USA,45.54532,-94.05834 295 | STL,St. Louis International Airport at Lambert Field,St Louis,MO,USA,38.74769,-90.35999 296 | STT,Cyril E. King Airport,Charlotte Amalie,VI,USA,18.33731,-64.97336 297 | STX,Henry E. Rohlsen Airport,Christiansted,VI,USA,17.70189,-64.79856 298 | SUN,Friedman Memorial Airport,Hailey,ID,USA,43.50484,-114.29659 299 | SUX,Sioux Gateway Airport,Sioux City,IA,USA,42.40260,-96.38437 300 | SWF,Stewart International Airport,Newburgh,NY,USA,41.50409,-74.10484 301 | SYR,Syracuse Hancock International Airport,Syracuse,NY,USA,43.11119,-76.10631 302 | TLH,Tallahassee International Airport,Tallahassee,FL,USA,30.39653,-84.35033 303 | TOL,Toledo Express Airport,Toledo,OH,USA,41.58681,-83.80783 304 | TPA,Tampa International Airport,Tampa,FL,USA,27.97547,-82.53325 305 | TRI,Tri-Cities Regional Airport,Bristol,TN,USA,36.47521,-82.40742 306 | TTN,Trenton Mercer Airport,Trenton,NJ,USA,40.27669,-74.81347 307 | TUL,Tulsa International Airport,Tulsa,OK,USA,36.19837,-95.88824 308 | TUS,Tucson International Airport,Tucson,AZ,USA,32.11608,-110.94103 309 | TVC,Cherry Capital Airport,Traverse City,MI,USA,44.74144,-85.58224 310 | TWF,Magic Valley Regional Airport (Joslin Field),Twin Falls,ID,USA,42.48180,-114.48774 311 | TXK,Texarkana Regional Airport (Webb Field),Texarkana,AR,USA,33.45371,-93.99102 312 | TYR,Tyler Pounds Regional Airport,Tyler,TX,USA,32.35414,-95.40239 313 | TYS,McGhee Tyson Airport,Knoxville,TN,USA,35.81249,-83.99286 314 | UST,Northeast Florida Regional Airport (St. Augustine Airport),St. Augustine,FL,USA,, 315 | VEL,Valdez Airport,Vernal,UT,USA,40.44090,-109.50992 316 | VLD,Valdosta Regional Airport,Valdosta,GA,USA,30.78250,-83.27672 317 | VPS,Destin-Fort Walton Beach Airport/Eglin AFB,Valparaiso,FL,USA,30.48325,-86.52540 318 | WRG,Wrangell Airport,Wrangell,AK,USA,56.48433,-132.36982 319 | WYS,Westerly State Airport,West Yellowstone,MT,USA,44.68840,-111.11764 320 | XNA,Northwest Arkansas Regional Airport,Fayetteville/Springdale/Rogers,AR,USA,36.28187,-94.30681 321 | YAK,Yakutat Airport,Yakutat,AK,USA,59.50336,-139.66023 322 | YUM,Yuma International Airport,Yuma,AZ,USA,32.65658,-114.60597 323 | -------------------------------------------------------------------------------- /aggregations-and-analytics/09_cdc_materialized_view/docker/client-image/LICENSE: -------------------------------------------------------------------------------- 1 | 2 | Apache License 3 | Version 2.0, January 2004 4 | http://www.apache.org/licenses/ 5 | 6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 7 | 8 | 1. Definitions. 9 | 10 | "License" shall mean the terms and conditions for use, reproduction, 11 | and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by 14 | the copyright owner that is granting the License. 15 | 16 | "Legal Entity" shall mean the union of the acting entity and all 17 | other entities that control, are controlled by, or are under common 18 | control with that entity. For the purposes of this definition, 19 | "control" means (i) the power, direct or indirect, to cause the 20 | direction or management of such entity, whether by contract or 21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 22 | outstanding shares, or (iii) beneficial ownership of such entity. 23 | 24 | "You" (or "Your") shall mean an individual or Legal Entity 25 | exercising permissions granted by this License. 26 | 27 | "Source" form shall mean the preferred form for making modifications, 28 | including but not limited to software source code, documentation 29 | source, and configuration files. 30 | 31 | "Object" form shall mean any form resulting from mechanical 32 | transformation or translation of a Source form, including but 33 | not limited to compiled object code, generated documentation, 34 | and conversions to other media types. 35 | 36 | "Work" shall mean the work of authorship, whether in Source or 37 | Object form, made available under the License, as indicated by a 38 | copyright notice that is included in or attached to the work 39 | (an example is provided in the Appendix below). 40 | 41 | "Derivative Works" shall mean any work, whether in Source or Object 42 | form, that is based on (or derived from) the Work and for which the 43 | editorial revisions, annotations, elaborations, or other modifications 44 | represent, as a whole, an original work of authorship. For the purposes 45 | of this License, Derivative Works shall not include works that remain 46 | separable from, or merely link (or bind by name) to the interfaces of, 47 | the Work and Derivative Works thereof. 48 | 49 | "Contribution" shall mean any work of authorship, including 50 | the original version of the Work and any modifications or additions 51 | to that Work or Derivative Works thereof, that is intentionally 52 | submitted to Licensor for inclusion in the Work by the copyright owner 53 | or by an individual or Legal Entity authorized to submit on behalf of 54 | the copyright owner. For the purposes of this definition, "submitted" 55 | means any form of electronic, verbal, or written communication sent 56 | to the Licensor or its representatives, including but not limited to 57 | communication on electronic mailing lists, source code control systems, 58 | and issue tracking systems that are managed by, or on behalf of, the 59 | Licensor for the purpose of discussing and improving the Work, but 60 | excluding communication that is conspicuously marked or otherwise 61 | designated in writing by the copyright owner as "Not a Contribution." 62 | 63 | "Contributor" shall mean Licensor and any individual or Legal Entity 64 | on behalf of whom a Contribution has been received by Licensor and 65 | subsequently incorporated within the Work. 66 | 67 | 2. Grant of Copyright License. Subject to the terms and conditions of 68 | this License, each Contributor hereby grants to You a perpetual, 69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 70 | copyright license to reproduce, prepare Derivative Works of, 71 | publicly display, publicly perform, sublicense, and distribute the 72 | Work and such Derivative Works in Source or Object form. 73 | 74 | 3. Grant of Patent License. Subject to the terms and conditions of 75 | this License, each Contributor hereby grants to You a perpetual, 76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 77 | (except as stated in this section) patent license to make, have made, 78 | use, offer to sell, sell, import, and otherwise transfer the Work, 79 | where such license applies only to those patent claims licensable 80 | by such Contributor that are necessarily infringed by their 81 | Contribution(s) alone or by combination of their Contribution(s) 82 | with the Work to which such Contribution(s) was submitted. If You 83 | institute patent litigation against any entity (including a 84 | cross-claim or counterclaim in a lawsuit) alleging that the Work 85 | or a Contribution incorporated within the Work constitutes direct 86 | or contributory patent infringement, then any patent licenses 87 | granted to You under this License for that Work shall terminate 88 | as of the date such litigation is filed. 89 | 90 | 4. Redistribution. You may reproduce and distribute copies of the 91 | Work or Derivative Works thereof in any medium, with or without 92 | modifications, and in Source or Object form, provided that You 93 | meet the following conditions: 94 | 95 | (a) You must give any other recipients of the Work or 96 | Derivative Works a copy of this License; and 97 | 98 | (b) You must cause any modified files to carry prominent notices 99 | stating that You changed the files; and 100 | 101 | (c) You must retain, in the Source form of any Derivative Works 102 | that You distribute, all copyright, patent, trademark, and 103 | attribution notices from the Source form of the Work, 104 | excluding those notices that do not pertain to any part of 105 | the Derivative Works; and 106 | 107 | (d) If the Work includes a "NOTICE" text file as part of its 108 | distribution, then any Derivative Works that You distribute must 109 | include a readable copy of the attribution notices contained 110 | within such NOTICE file, excluding those notices that do not 111 | pertain to any part of the Derivative Works, in at least one 112 | of the following places: within a NOTICE text file distributed 113 | as part of the Derivative Works; within the Source form or 114 | documentation, if provided along with the Derivative Works; or, 115 | within a display generated by the Derivative Works, if and 116 | wherever such third-party notices normally appear. The contents 117 | of the NOTICE file are for informational purposes only and 118 | do not modify the License. You may add Your own attribution 119 | notices within Derivative Works that You distribute, alongside 120 | or as an addendum to the NOTICE text from the Work, provided 121 | that such additional attribution notices cannot be construed 122 | as modifying the License. 123 | 124 | You may add Your own copyright statement to Your modifications and 125 | may provide additional or different license terms and conditions 126 | for use, reproduction, or distribution of Your modifications, or 127 | for any such Derivative Works as a whole, provided Your use, 128 | reproduction, and distribution of the Work otherwise complies with 129 | the conditions stated in this License. 130 | 131 | 5. Submission of Contributions. Unless You explicitly state otherwise, 132 | any Contribution intentionally submitted for inclusion in the Work 133 | by You to the Licensor shall be under the terms and conditions of 134 | this License, without any additional terms or conditions. 135 | Notwithstanding the above, nothing herein shall supersede or modify 136 | the terms of any separate license agreement you may have executed 137 | with Licensor regarding such Contributions. 138 | 139 | 6. Trademarks. This License does not grant permission to use the trade 140 | names, trademarks, service marks, or product names of the Licensor, 141 | except as required for reasonable and customary use in describing the 142 | origin of the Work and reproducing the content of the NOTICE file. 143 | 144 | 7. Disclaimer of Warranty. Unless required by applicable law or 145 | agreed to in writing, Licensor provides the Work (and each 146 | Contributor provides its Contributions) on an "AS IS" BASIS, 147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 148 | implied, including, without limitation, any warranties or conditions 149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 150 | PARTICULAR PURPOSE. You are solely responsible for determining the 151 | appropriateness of using or redistributing the Work and assume any 152 | risks associated with Your exercise of permissions under this License. 153 | 154 | 8. Limitation of Liability. In no event and under no legal theory, 155 | whether in tort (including negligence), contract, or otherwise, 156 | unless required by applicable law (such as deliberate and grossly 157 | negligent acts) or agreed to in writing, shall any Contributor be 158 | liable to You for damages, including any direct, indirect, special, 159 | incidental, or consequential damages of any character arising as a 160 | result of this License or out of the use or inability to use the 161 | Work (including but not limited to damages for loss of goodwill, 162 | work stoppage, computer failure or malfunction, or any and all 163 | other commercial damages or losses), even if such Contributor 164 | has been advised of the possibility of such damages. 165 | 166 | 9. Accepting Warranty or Additional Liability. While redistributing 167 | the Work or Derivative Works thereof, You may choose to offer, 168 | and charge a fee for, acceptance of support, warranty, indemnity, 169 | or other liability obligations and/or rights consistent with this 170 | License. However, in accepting such obligations, You may act only 171 | on Your own behalf and on Your sole responsibility, not on behalf 172 | of any other Contributor, and only if You agree to indemnify, 173 | defend, and hold each Contributor harmless for any liability 174 | incurred by, or claims asserted against, such Contributor by reason 175 | of your accepting any such warranty or additional liability. 176 | 177 | END OF TERMS AND CONDITIONS 178 | 179 | APPENDIX: How to apply the Apache License to your work. 180 | 181 | To apply the Apache License to your work, attach the following 182 | boilerplate notice, with the fields enclosed by brackets "[]" 183 | replaced with your own identifying information. (Don't include 184 | the brackets!) The text should be enclosed in the appropriate 185 | comment syntax for the file format. We also recommend that a 186 | file or class name and description of purpose be included on the 187 | same "printed page" as the copyright notice for easier 188 | identification within third-party archives. 189 | 190 | Copyright [yyyy] [name of copyright owner] 191 | 192 | Licensed under the Apache License, Version 2.0 (the "License"); 193 | you may not use this file except in compliance with the License. 194 | You may obtain a copy of the License at 195 | 196 | http://www.apache.org/licenses/LICENSE-2.0 197 | 198 | Unless required by applicable law or agreed to in writing, software 199 | distributed under the License is distributed on an "AS IS" BASIS, 200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 201 | See the License for the specific language governing permissions and 202 | limitations under the License. 203 | 204 | This distribution has a binary dependency on jersey, which is available under the CDDL 205 | License as described below. 206 | 207 | COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL - Version 1.1) 208 | 1. Definitions. 209 | 1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. 210 | 211 | 1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. 212 | 213 | 1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. 214 | 215 | 1.4. “Executable” means the Covered Software in any form other than Source Code. 216 | 217 | 1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. 218 | 219 | 1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. 220 | 221 | 1.7. “License” means this document. 222 | 223 | 1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. 224 | 225 | 1.9. “Modifications” means the Source Code and Executable form of any of the following: 226 | 227 | A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; 228 | 229 | B. Any new file that contains any part of the Original Software or previous Modification; or 230 | 231 | C. Any new file that is contributed or otherwise made available under the terms of this License. 232 | 233 | 1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. 234 | 235 | 1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. 236 | 237 | 1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. 238 | 239 | 1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. 240 | 241 | 2. License Grants. 242 | 2.1. The Initial Developer Grant. 243 | 244 | Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: 245 | 246 | (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and 247 | 248 | (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). 249 | 250 | (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. 251 | 252 | (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. 253 | 254 | 2.2. Contributor Grant. 255 | 256 | Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: 257 | 258 | (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and 259 | 260 | (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). 261 | 262 | (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. 263 | 264 | (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. 265 | 266 | 3. Distribution Obligations. 267 | 3.1. Availability of Source Code. 268 | 269 | Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. 270 | 271 | 3.2. Modifications. 272 | 273 | The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. 274 | 275 | 3.3. Required Notices. 276 | 277 | You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. 278 | 279 | 3.4. Application of Additional Terms. 280 | 281 | You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients’ rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. 282 | 283 | 3.5. Distribution of Executable Versions. 284 | 285 | You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient’s rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. 286 | 287 | 3.6. Larger Works. 288 | 289 | You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. 290 | 291 | 4. Versions of the License. 292 | 4.1. New Versions. 293 | 294 | Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. 295 | 296 | 4.2. Effect of New Versions. 297 | 298 | You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. 299 | 300 | 4.3. Modified Versions. 301 | 302 | When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. 303 | 304 | 5. DISCLAIMER OF WARRANTY. 305 | COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. 306 | 307 | 6. TERMINATION. 308 | 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. 309 | 310 | 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. 311 | 312 | 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. 313 | 314 | 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. 315 | 316 | 7. LIMITATION OF LIABILITY. 317 | UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY’S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. 318 | 319 | 8. U.S. GOVERNMENT END USERS. 320 | The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. 321 | 322 | 9. MISCELLANEOUS. 323 | This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction’s conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys’ fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. 324 | 325 | 10. RESPONSIBILITY FOR CLAIMS. 326 | As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. 327 | 328 | NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) 329 | 330 | The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. 331 | --------------------------------------------------------------------------------