├── FlinkExperiments ├── .gitignore ├── img │ └── screenshot.jpg ├── pom.xml ├── scripts │ └── station_csv_to_sql.ps1 ├── sql │ ├── .gitignore │ ├── 01_Schemas │ │ └── schema_sample.sql │ ├── 02_Tables │ │ └── tables_sample.sql │ ├── 03_Keys │ │ └── keys_sample.sql │ ├── 05_Security │ │ └── security_sample.sql │ ├── 06_Data │ │ └── data_sample_stations.sql │ ├── create_database.bat │ └── create_database.sh └── src │ └── main │ ├── java │ ├── app │ │ ├── WeatherDataStreamingExample.java │ │ └── cep │ │ │ ├── WeatherDataComplexEventProcessingExample.java │ │ │ ├── WeatherDataComplexEventProcessingExample2.java │ │ │ ├── mocks │ │ │ └── SevereHeatWarningSourceFunction.java │ │ │ └── model │ │ │ ├── IWarning.java │ │ │ ├── IWarningPattern.java │ │ │ ├── patterns │ │ │ ├── temperature │ │ │ │ ├── ExcessiveHeatWarningPattern.java │ │ │ │ ├── ExtremeColdWarningPattern.java │ │ │ │ └── SevereHeatWarningPattern.java │ │ │ └── wind │ │ │ │ ├── ExtremeWindWarningPattern.java │ │ │ │ └── HighWindWarningPattern.java │ │ │ └── warnings │ │ │ ├── temperature │ │ │ ├── ExcessiveHeatWarning.java │ │ │ ├── ExtremeColdWarning.java │ │ │ └── SevereHeatWarning.java │ │ │ └── wind │ │ │ ├── ExtremeWindWarning.java │ │ │ └── HighWindWarning.java │ ├── csv │ │ ├── mapping │ │ │ ├── LocalWeatherDataMapper.java │ │ │ └── StationMapper.java │ │ ├── model │ │ │ ├── LocalWeatherData.java │ │ │ └── Station.java │ │ ├── parser │ │ │ └── Parsers.java │ │ └── sorting │ │ │ └── PrepareWeatherData.java │ ├── elastic │ │ ├── converter │ │ │ └── LocalWeatherDataConverter.java │ │ ├── mapping │ │ │ └── LocalWeatherDataMapper.java │ │ └── model │ │ │ ├── GeoLocation.java │ │ │ ├── LocalWeatherData.java │ │ │ └── Station.java │ ├── model │ │ ├── GeoLocation.java │ │ ├── LocalWeatherData.java │ │ └── Station.java │ ├── pgsql │ │ ├── connection │ │ │ └── PooledConnectionFactory.java │ │ ├── converter │ │ │ └── LocalWeatherDataConverter.java │ │ ├── mapping │ │ │ └── LocalWeatherDataMapping.java │ │ └── model │ │ │ └── LocalWeatherData.java │ ├── stream │ │ ├── sinks │ │ │ ├── elastic │ │ │ │ ├── BaseElasticSearchSink.java │ │ │ │ └── LocalWeatherDataElasticSearchSink.java │ │ │ └── pgsql │ │ │ │ ├── BasePostgresSink.java │ │ │ │ └── LocalWeatherDataPostgresSink.java │ │ └── sources │ │ │ ├── csv │ │ │ ├── LocalWeatherDataSourceFunction.java │ │ │ └── converter │ │ │ │ └── LocalWeatherDataConverter.java │ │ │ └── periodic │ │ │ └── PeriodicEmittingDataSourceFunction.java │ └── utils │ │ ├── DateUtilities.java │ │ └── StringUtils.java │ └── resources │ └── logback.xml ├── LICENSE └── README.md /FlinkExperiments/.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | target 3 | elasticutils-*-javadoc.jar 4 | elasticutils-*-sources.jar 5 | *.asc 6 | stderr.log 7 | stdout.log 8 | FlinkExperiment.iml -------------------------------------------------------------------------------- /FlinkExperiments/img/screenshot.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/bytefish/FlinkExperiments/4b6d30134e542eadb46d0b9fd4e6bae9d28e91bd/FlinkExperiments/img/screenshot.jpg -------------------------------------------------------------------------------- /FlinkExperiments/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | de.bytefish 8 | flinkexperiment 9 | 1.0 10 | 11 | 12 | 13 | 14 | org.apache.maven.plugins 15 | maven-compiler-plugin 16 | 17 | 1.8 18 | 1.8 19 | 20 | 21 | 22 | 23 | 24 | 25 | UTF-8 26 | 1.6.2 27 | 3.3 28 | 1.2 29 | 1.0 30 | 2.1.3 31 | 32 | 33 | 34 | 35 | 36 | org.apache.flink 37 | flink-java 38 | ${flink.version} 39 | 40 | 41 | 42 | org.apache.flink 43 | flink-streaming-java_2.11 44 | ${flink.version} 45 | 46 | 47 | org.apache.flink 48 | flink-clients_2.11 49 | ${flink.version} 50 | 51 | 52 | 53 | org.apache.flink 54 | flink-cep_2.11 55 | ${flink.version} 56 | 57 | 58 | 59 | ch.qos.logback 60 | logback-classic 61 | 1.2.3 62 | 63 | 64 | 65 | de.bytefish 66 | jtinycsvparser 67 | ${jtinycsvparser.version} 68 | 69 | 70 | 71 | org.elasticsearch 72 | elasticsearch 73 | 6.5.1 74 | 75 | 76 | 77 | com.fasterxml.jackson.core 78 | jackson-annotations 79 | [2.9.8,) 80 | 81 | 82 | 83 | com.fasterxml.jackson.core 84 | jackson-databind 85 | [2.9.8,) 86 | 87 | 88 | 89 | de.bytefish.elasticutils 90 | elasticutils-core 91 | ${elasticutils.version} 92 | 93 | 94 | 95 | de.bytefish.elasticutils 96 | elasticutils-elasticsearch6 97 | ${elasticutils.version} 98 | 99 | 100 | 101 | de.bytefish 102 | pgbulkinsert 103 | ${pgbulkinsert.version} 104 | 105 | 106 | 107 | io.reactivex.rxjava2 108 | rxjava 109 | ${rxjava2.version} 110 | 111 | 112 | 113 | org.apache.commons 114 | commons-dbcp2 115 | 2.0.1 116 | 117 | 118 | 119 | junit 120 | junit 121 | 4.13.1 122 | test 123 | 124 | 125 | 126 | 127 | -------------------------------------------------------------------------------- /FlinkExperiments/scripts/station_csv_to_sql.ps1: -------------------------------------------------------------------------------- 1 | param ( 2 | [string]$in_filename, 3 | [string]$out_filename 4 | ) 5 | 6 | function GetSqlCompatibleValue([string] $val) 7 | { 8 | if([string]::IsNullOrWhiteSpace($val)) { 9 | return "NULL" 10 | } 11 | return $val.Replace("'", "''"); 12 | } 13 | 14 | 15 | 16 | Import-Csv $in_filename -Delimiter "|" | Foreach-Object{ 17 | 18 | $line = 19 | @" 20 | `nINSERT INTO sample.station(wban, name, state, location, latitude, longitude, ground_height, station_height, timeZone) 21 | SELECT '{0}', '{1}', '{2}', '{3}', {4}, {5}, {6}, {7}, {8} 22 | WHERE NOT EXISTS (SELECT 1 FROM sample.station WHERE wban='{0}'); 23 | "@ -f (GetSqlCompatibleValue $_.WBAN), (GetSqlCompatibleValue $_.Name), (GetSqlCompatibleValue $_.State), (GetSqlCompatibleValue $_.Location), (GetSqlCompatibleValue $_.Latitude), (GetSqlCompatibleValue $_.Longitude), (GetSqlCompatibleValue $_.GroundHeight), (GetSqlCompatibleValue $_.StationHeight), (GetSqlCompatibleValue $_.TimeZone) 24 | 25 | $line|Out-File $out_filename -Append 26 | 27 | } 28 | -------------------------------------------------------------------------------- /FlinkExperiments/sql/.gitignore: -------------------------------------------------------------------------------- 1 | *.log -------------------------------------------------------------------------------- /FlinkExperiments/sql/01_Schemas/schema_sample.sql: -------------------------------------------------------------------------------- 1 | DO $$ 2 | BEGIN 3 | 4 | IF NOT EXISTS (SELECT 1 FROM information_schema.schemata WHERE schema_name = 'sample') THEN 5 | 6 | CREATE SCHEMA sample; 7 | 8 | END IF; 9 | 10 | END; 11 | $$; -------------------------------------------------------------------------------- /FlinkExperiments/sql/02_Tables/tables_sample.sql: -------------------------------------------------------------------------------- 1 | DO $$ 2 | BEGIN 3 | 4 | IF NOT EXISTS ( 5 | SELECT 1 6 | FROM information_schema.tables 7 | WHERE table_schema = 'sample' 8 | AND table_name = 'station' 9 | ) THEN 10 | 11 | CREATE TABLE sample.station 12 | ( 13 | station_id SERIAL PRIMARY KEY, 14 | wban VARCHAR(255) NOT NULL, 15 | name VARCHAR(255) NOT NULL, 16 | state VARCHAR(255), 17 | location VARCHAR(255), 18 | latitude REAL NOT NULL, 19 | longitude REAL NOT NULL, 20 | ground_height SMALLINT, 21 | station_height SMALLINT, 22 | TimeZone SMALLINT 23 | ); 24 | 25 | END IF; 26 | 27 | IF NOT EXISTS ( 28 | SELECT 1 29 | FROM information_schema.tables 30 | WHERE table_schema = 'sample' 31 | AND table_name = 'weather_data' 32 | ) THEN 33 | 34 | CREATE TABLE sample.weather_data 35 | ( 36 | wban VARCHAR(255), 37 | dateTime TIMESTAMP, 38 | temperature REAL, 39 | windSpeed REAL, 40 | stationPressure REAL, 41 | skyCondition VARCHAR(255) 42 | ); 43 | 44 | END IF; 45 | 46 | END; 47 | $$; -------------------------------------------------------------------------------- /FlinkExperiments/sql/03_Keys/keys_sample.sql: -------------------------------------------------------------------------------- 1 | DO $$ 2 | BEGIN 3 | 4 | IF NOT EXISTS (SELECT 1 FROM pg_constraint WHERE conname = 'uk_station_wban') THEN 5 | ALTER TABLE sample.station 6 | ADD CONSTRAINT uk_station_wban 7 | UNIQUE (wban); 8 | END IF; 9 | 10 | END; 11 | $$; -------------------------------------------------------------------------------- /FlinkExperiments/sql/05_Security/security_sample.sql: -------------------------------------------------------------------------------- 1 | DO $$ 2 | BEGIN 3 | 4 | REVOKE ALL ON sample.station FROM public; 5 | REVOKE ALL ON sample.weather_data FROM public; 6 | 7 | END; 8 | $$; -------------------------------------------------------------------------------- /FlinkExperiments/sql/create_database.bat: -------------------------------------------------------------------------------- 1 | @echo off 2 | 3 | :: Copyright (c) Philipp Wagner. All rights reserved. 4 | :: Licensed under the MIT license. See LICENSE file in the project root for full license information. 5 | 6 | set PGSQL_EXECUTABLE="C:\Program Files\PostgreSQL\9.4\bin\psql.exe" 7 | set STDOUT=stdout.log 8 | set STDERR=stderr.log 9 | set LOGFILE=query_output.log 10 | 11 | set HostName=localhost 12 | set PortNumber=5432 13 | set DatabaseName=sampledb 14 | set UserName=philipp 15 | set Password= 16 | 17 | call :AskQuestionWithYdefault "Use Host (%HostName%) Port (%PortNumber%) [Y,n]?" reply_ 18 | if /i [%reply_%] NEQ [y] ( 19 | set /p HostName="Enter HostName: " 20 | set /p PortNumber="Enter Port: " 21 | ) 22 | 23 | call :AskQuestionWithYdefault "Use Database (%DatabaseName%) [Y,n]?" reply_ 24 | if /i [%reply_%] NEQ [y] ( 25 | set /p ServerName="Enter Database: " 26 | ) 27 | 28 | call :AskQuestionWithYdefault "Use User (%UserName%) [Y,n]?" reply_ 29 | if /i [%reply_%] NEQ [y] ( 30 | set /p UserName="Enter User: " 31 | ) 32 | 33 | set /p PGPASSWORD="Password: " 34 | 35 | 1>%STDOUT% 2>%STDERR% ( 36 | 37 | :: Schemas 38 | %PGSQL_EXECUTABLE% -h %HostName% -p %PortNumber% -d %DatabaseName% -U %UserName% < 01_Schemas/schema_sample.sql -L %LOGFILE% 39 | 40 | :: Tables 41 | %PGSQL_EXECUTABLE% -h %HostName% -p %PortNumber% -d %DatabaseName% -U %UserName% < 02_Tables/tables_sample.sql -L %LOGFILE% 42 | 43 | :: Keys 44 | %PGSQL_EXECUTABLE% -h %HostName% -p %PortNumber% -d %DatabaseName% -U %UserName% < 03_Keys/keys_sample.sql -L %LOGFILE% 45 | 46 | :: Security 47 | %PGSQL_EXECUTABLE% -h %HostName% -p %PortNumber% -d %DatabaseName% -U %UserName% < 05_Security/security_sample.sql -L %LOGFILE% 48 | 49 | :: Data 50 | %PGSQL_EXECUTABLE% -h %HostName% -p %PortNumber% -d %DatabaseName% -U %UserName% < 06_Data/data_sample_stations.sql -L %LOGFILE% 51 | ) 52 | 53 | goto :end 54 | 55 | :: The question as a subroutine 56 | :AskQuestionWithYdefault 57 | setlocal enableextensions 58 | :_asktheyquestionagain 59 | set return_= 60 | set ask_= 61 | set /p ask_="%~1" 62 | if "%ask_%"=="" set return_=y 63 | if /i "%ask_%"=="Y" set return_=y 64 | if /i "%ask_%"=="n" set return_=n 65 | if not defined return_ goto _asktheyquestionagain 66 | endlocal & set "%2=%return_%" & goto :EOF 67 | 68 | :end 69 | pause -------------------------------------------------------------------------------- /FlinkExperiments/sql/create_database.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | # Copyright (c) Philipp Wagner. All rights reserved. 4 | # Licensed under the MIT license. See LICENSE file in the project root for full license information. 5 | 6 | 7 | function ask_yes_or_no() { 8 | read -p "$1 ([y]es or [N]o): " 9 | case $(echo $REPLY | tr '[A-Z]' '[a-z]') in 10 | y|yes) echo "yes" ;; 11 | *) echo "no" ;; 12 | esac 13 | } 14 | 15 | STDOUT=stdout.log 16 | STDERR=stderr.log 17 | LOGFILE=query_output.log 18 | 19 | HostName=localhost 20 | PortNumber=5432 21 | DatabaseName=sampledb 22 | UserName=philipp 23 | 24 | if [[ "no" == $(ask_yes_or_no "Use Host ($HostName) Port ($PortNumber)") ]] 25 | then 26 | read -p "Enter HostName: " HostName 27 | read -p "Enter Port: " PortNumber 28 | fi 29 | 30 | if [[ "no" == $(ask_yes_or_no "Use Database ($DatabaseName)") ]] 31 | then 32 | read -p "Enter Database: " ServerName 33 | fi 34 | 35 | if [[ "no" == $(ask_yes_or_no "Use User ($UserName)") ]] 36 | then 37 | read -p "Enter User: " UserName 38 | fi 39 | 40 | read -p "Password: " PGPASSWORD 41 | 42 | # Schemas 43 | psql -h $HostName -p $PortNumber -d $DatabaseName -U $UserName < 01_Schemas/schema_sample.sql -L $LOGFILE 1>$STDOUT 2>$STDERR 44 | 45 | # Tables 46 | psql -h $HostName -p $PortNumber -d $DatabaseName -U $UserName < 02_Tables/tables_sample.sql -L $LOGFILE 1>>$STDOUT 2>>$STDERR 47 | 48 | # Keys 49 | psql -h $HostName -p $PortNumber -d $DatabaseName -U $UserName < 03_Keys/keys_sample.sql -L $LOGFILE 1>>$STDOUT 2>>$STDERR 50 | 51 | # Security 52 | psql -h $HostName -p $PortNumber -d $DatabaseName -U $UserName < 05_Security/security_sample.sql -L $LOGFILE 1>>$STDOUT 2>>$STDERR 53 | 54 | # Data 55 | psql -h $HostName -p $PortNumber -d $DatabaseName -U $UserName < 06_Data/data_sample_stations.sql -L $LOGFILE 1>>$STDOUT 2>>$STDERR -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/WeatherDataStreamingExample.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app; 5 | 6 | import model.LocalWeatherData; 7 | import org.apache.flink.api.common.functions.FilterFunction; 8 | import org.apache.flink.api.common.functions.MapFunction; 9 | import org.apache.flink.api.java.functions.KeySelector; 10 | import org.apache.flink.streaming.api.TimeCharacteristic; 11 | import org.apache.flink.streaming.api.datastream.DataStream; 12 | import org.apache.flink.streaming.api.datastream.KeyedStream; 13 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 14 | import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; 15 | import org.apache.flink.streaming.api.windowing.time.Time; 16 | import stream.sinks.elastic.LocalWeatherDataElasticSearchSink; 17 | import stream.sinks.pgsql.LocalWeatherDataPostgresSink; 18 | import stream.sources.csv.LocalWeatherDataSourceFunction; 19 | import utils.DateUtilities; 20 | 21 | import java.net.URI; 22 | import java.time.ZoneOffset; 23 | import java.util.Date; 24 | 25 | public class WeatherDataStreamingExample { 26 | 27 | public static void main(String[] args) throws Exception { 28 | 29 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 30 | 31 | // Use the Measurement Timestamp of the Event: 32 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 33 | 34 | // Path to read the CSV data from: 35 | final String csvStationDataFilePath = "D:\\datasets\\201503station.txt"; 36 | final String csvLocalWeatherDataFilePath = "D:\\datasets\\201503hourly_sorted.txt"; 37 | 38 | // Add the CSV Data Source and assign the Measurement Timestamp: 39 | DataStream localWeatherDataDataStream = env 40 | .addSource(new LocalWeatherDataSourceFunction(csvStationDataFilePath, csvLocalWeatherDataFilePath)) 41 | .assignTimestampsAndWatermarks(new AscendingTimestampExtractor() { 42 | @Override 43 | public long extractAscendingTimestamp(LocalWeatherData localWeatherData) { 44 | Date measurementTime = DateUtilities.from(localWeatherData.getDate(), localWeatherData.getTime(), ZoneOffset.ofHours(0)); 45 | 46 | return measurementTime.getTime(); 47 | } 48 | }); 49 | 50 | // First build a KeyedStream over the Data with LocalWeather: 51 | KeyedStream localWeatherDataByStation = localWeatherDataDataStream 52 | // Filte for Non-Null Temperature Values, because we might have missing data: 53 | .filter(new FilterFunction() { 54 | @Override 55 | public boolean filter(LocalWeatherData localWeatherData) throws Exception { 56 | return localWeatherData.getTemperature() != null; 57 | } 58 | }) 59 | // Now create the keyed stream by the Station WBAN identifier: 60 | .keyBy(new KeySelector() { 61 | @Override 62 | public String getKey(LocalWeatherData localWeatherData) throws Exception { 63 | return localWeatherData.getStation().getWban(); 64 | } 65 | }); 66 | 67 | // Now take the Maximum Temperature per day from the KeyedStream: 68 | DataStream maxTemperaturePerDay = 69 | localWeatherDataByStation 70 | // Use non-overlapping tumbling window with 1 day length: 71 | .timeWindow(Time.days(1)) 72 | // And use the maximum temperature: 73 | .maxBy("temperature"); 74 | 75 | // Converts the general stream into the Elasticsearch specific representation with JsonAttributes: 76 | DataStream elasticDailyMaxTemperature = maxTemperaturePerDay 77 | .map(new MapFunction() { 78 | @Override 79 | public elastic.model.LocalWeatherData map(model.LocalWeatherData localWeatherData) throws Exception { 80 | return elastic.converter.LocalWeatherDataConverter.convert(localWeatherData); 81 | } 82 | }); 83 | 84 | // Converts the general stream into the Postgres-specific representation: 85 | DataStream pgsqlDailyMaxTemperature = maxTemperaturePerDay 86 | .map(new MapFunction() { 87 | @Override 88 | public pgsql.model.LocalWeatherData map(model.LocalWeatherData localWeatherData) throws Exception { 89 | return pgsql.converter.LocalWeatherDataConverter.convert(localWeatherData); 90 | } 91 | }); 92 | 93 | 94 | // Add a new ElasticSearch Sink: 95 | elasticDailyMaxTemperature.addSink(new LocalWeatherDataElasticSearchSink("127.0.0.1", 9300, 100)); 96 | 97 | // Add a new Postgres Sink: 98 | //pgsqlDailyMaxTemperature.addSink(new LocalWeatherDataPostgresSink(URI.create("postgres://philipp:test_pwd@127.0.0.1:5432/sampledb"), 1000)); 99 | 100 | // Finally execute the Stream: 101 | env.execute("Max Temperature By Day example"); 102 | } 103 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/WeatherDataComplexEventProcessingExample.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep; 5 | 6 | import app.cep.model.IWarning; 7 | import app.cep.model.IWarningPattern; 8 | import app.cep.model.patterns.temperature.SevereHeatWarningPattern; 9 | import app.cep.model.warnings.temperature.SevereHeatWarning; 10 | import model.LocalWeatherData; 11 | import org.apache.flink.api.common.functions.FilterFunction; 12 | import org.apache.flink.api.java.functions.KeySelector; 13 | import org.apache.flink.api.java.typeutils.GenericTypeInfo; 14 | import org.apache.flink.cep.CEP; 15 | import org.apache.flink.cep.PatternSelectFunction; 16 | import org.apache.flink.cep.PatternStream; 17 | import org.apache.flink.streaming.api.TimeCharacteristic; 18 | import org.apache.flink.streaming.api.datastream.DataStream; 19 | import org.apache.flink.streaming.api.datastream.KeyedStream; 20 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 21 | import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; 22 | import org.apache.flink.streaming.api.windowing.time.Time; 23 | import stream.sources.csv.LocalWeatherDataSourceFunction; 24 | import utils.DateUtilities; 25 | 26 | import java.time.ZoneOffset; 27 | import java.util.Date; 28 | import java.util.List; 29 | import java.util.Map; 30 | 31 | public class WeatherDataComplexEventProcessingExample { 32 | 33 | public static void main(String[] args) throws Exception { 34 | 35 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 36 | 37 | // Use the Measurement Timestamp of the Event: 38 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 39 | 40 | // Path to read the CSV data from: 41 | final String csvStationDataFilePath = "C:\\Users\\philipp\\Downloads\\csv\\201503station.txt"; 42 | final String csvLocalWeatherDataFilePath = "C:\\Users\\philipp\\Downloads\\csv\\201503hourly_sorted.txt"; 43 | 44 | 45 | // Add the CSV Data Source and assign the Measurement Timestamp: 46 | DataStream localWeatherDataDataStream = env 47 | .addSource(new LocalWeatherDataSourceFunction(csvStationDataFilePath, csvLocalWeatherDataFilePath)) 48 | .assignTimestampsAndWatermarks(new AscendingTimestampExtractor() { 49 | @Override 50 | public long extractAscendingTimestamp(LocalWeatherData localWeatherData) { 51 | Date measurementTime = DateUtilities.from(localWeatherData.getDate(), localWeatherData.getTime(), ZoneOffset.ofHours(0)); 52 | 53 | return measurementTime.getTime(); 54 | } 55 | }); 56 | 57 | // First build a KeyedStream over the Data with LocalWeather: 58 | KeyedStream localWeatherDataByStation = localWeatherDataDataStream 59 | // Filter for Non-Null Temperature Values, because we might have missing data: 60 | .filter(new FilterFunction() { 61 | @Override 62 | public boolean filter(LocalWeatherData localWeatherData) throws Exception { 63 | return localWeatherData.getTemperature() != null; 64 | } 65 | }) 66 | // Now create the keyed stream by the Station WBAN identifier: 67 | .keyBy(new KeySelector() { 68 | @Override 69 | public String getKey(LocalWeatherData localWeatherData) throws Exception { 70 | return localWeatherData.getStation().getWban(); 71 | } 72 | }); 73 | 74 | // Now take the Maximum Temperature per day from the KeyedStream: 75 | DataStream maxTemperaturePerDay = 76 | localWeatherDataByStation 77 | // Use non-overlapping tumbling window with 1 day length: 78 | .timeWindow(Time.days(1)) 79 | // And use the maximum temperature: 80 | .maxBy("temperature"); 81 | 82 | // Now apply the SevereHeatWarningPattern on the Stream: 83 | DataStream warnings = toWarningStream(maxTemperaturePerDay, new SevereHeatWarningPattern()); 84 | 85 | // Print the warning to the Console for now: 86 | warnings.print(); 87 | 88 | // Finally execute the Stream: 89 | env.execute("CEP Weather Warning Example"); 90 | } 91 | 92 | private static DataStream toWarningStream(DataStream localWeatherDataDataStream, IWarningPattern warningPattern) { 93 | PatternStream tempPatternStream = CEP.pattern( 94 | localWeatherDataDataStream.keyBy(new KeySelector() { 95 | @Override 96 | public String getKey(LocalWeatherData localWeatherData) throws Exception { 97 | return localWeatherData.getStation().getWban(); 98 | } 99 | }), 100 | warningPattern.getEventPattern()); 101 | 102 | DataStream warnings = tempPatternStream.select(new PatternSelectFunction() { 103 | @Override 104 | public TWarningType select(Map> map) throws Exception { 105 | return warningPattern.create(map); 106 | } 107 | }, new GenericTypeInfo(warningPattern.getWarningTargetType())); 108 | 109 | return warnings; 110 | } 111 | 112 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/WeatherDataComplexEventProcessingExample2.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep; 5 | 6 | import app.cep.mocks.SevereHeatWarningSourceFunction; 7 | import app.cep.model.IWarning; 8 | import app.cep.model.IWarningPattern; 9 | import app.cep.model.patterns.temperature.SevereHeatWarningPattern; 10 | import app.cep.model.warnings.temperature.SevereHeatWarning; 11 | import model.LocalWeatherData; 12 | import org.apache.flink.api.common.functions.FilterFunction; 13 | import org.apache.flink.api.java.functions.KeySelector; 14 | import org.apache.flink.api.java.typeutils.GenericTypeInfo; 15 | import org.apache.flink.cep.CEP; 16 | import org.apache.flink.cep.PatternSelectFunction; 17 | import org.apache.flink.cep.PatternStream; 18 | import org.apache.flink.streaming.api.TimeCharacteristic; 19 | import org.apache.flink.streaming.api.datastream.DataStream; 20 | import org.apache.flink.streaming.api.datastream.KeyedStream; 21 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 22 | import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; 23 | import org.apache.flink.streaming.api.windowing.time.Time; 24 | import utils.DateUtilities; 25 | 26 | import java.time.ZoneOffset; 27 | import java.util.Date; 28 | import java.util.List; 29 | import java.util.Map; 30 | 31 | public class WeatherDataComplexEventProcessingExample2 { 32 | 33 | 34 | public static void main(String[] args) throws Exception { 35 | 36 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 37 | 38 | // Use the Measurement Timestamp of the Event: 39 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 40 | 41 | // Add the Data Source and assign the Measurement Timestamp: 42 | DataStream localWeatherDataDataStream = env 43 | .addSource(new SevereHeatWarningSourceFunction()) 44 | .assignTimestampsAndWatermarks(new AscendingTimestampExtractor() { 45 | @Override 46 | public long extractAscendingTimestamp(LocalWeatherData localWeatherData) { 47 | Date measurementTime = DateUtilities.from(localWeatherData.getDate(), localWeatherData.getTime(), ZoneOffset.ofHours(0)); 48 | 49 | return measurementTime.getTime(); 50 | } 51 | }); 52 | 53 | // First build a KeyedStream over the Data with LocalWeather: 54 | KeyedStream localWeatherDataByStation = localWeatherDataDataStream 55 | // Filter for Non-Null Temperature Values, because we might have missing data: 56 | .filter(new FilterFunction() { 57 | @Override 58 | public boolean filter(LocalWeatherData localWeatherData) throws Exception { 59 | return localWeatherData.getTemperature() != null; 60 | } 61 | }) 62 | // Now create the keyed stream by the Station WBAN identifier: 63 | .keyBy(new KeySelector() { 64 | @Override 65 | public String getKey(LocalWeatherData localWeatherData) throws Exception { 66 | return localWeatherData.getStation().getWban(); 67 | } 68 | }); 69 | 70 | // Now take the Maximum Temperature per day from the KeyedStream: 71 | DataStream maxTemperaturePerDay = 72 | localWeatherDataByStation 73 | // Use non-overlapping tumbling window with 1 day length: 74 | .timeWindow(Time.days(1)) 75 | // And use the maximum temperature: 76 | .maxBy("temperature"); 77 | 78 | // Now apply the SevereHeatWarningPattern on the Stream: 79 | DataStream warnings = toWarningStream(maxTemperaturePerDay, new SevereHeatWarningPattern()); 80 | 81 | // Print the warning to the Console for now: 82 | warnings.print(); 83 | 84 | // Finally execute the Stream: 85 | env.execute("CEP Weather Warning Example"); 86 | } 87 | 88 | private static DataStream toWarningStream(DataStream localWeatherDataDataStream, IWarningPattern warningPattern) { 89 | PatternStream tempPatternStream = CEP.pattern( 90 | localWeatherDataDataStream.keyBy(new KeySelector() { 91 | @Override 92 | public String getKey(LocalWeatherData localWeatherData) throws Exception { 93 | return localWeatherData.getStation().getWban(); 94 | } 95 | }), 96 | warningPattern.getEventPattern()); 97 | 98 | DataStream warnings = tempPatternStream.select(new PatternSelectFunction() { 99 | @Override 100 | public TWarningType select(Map> map) throws Exception { 101 | return warningPattern.create(map); 102 | } 103 | }, new GenericTypeInfo(warningPattern.getWarningTargetType())); 104 | 105 | return warnings; 106 | } 107 | 108 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/mocks/SevereHeatWarningSourceFunction.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.mocks; 5 | 6 | import model.GeoLocation; 7 | import model.LocalWeatherData; 8 | import model.Station; 9 | import stream.sources.periodic.PeriodicEmittingDataSourceFunction; 10 | 11 | import java.time.Duration; 12 | import java.time.LocalDate; 13 | import java.time.LocalTime; 14 | import java.util.Arrays; 15 | import java.util.List; 16 | 17 | /** 18 | * We build a Mock here with Fake Data. This Mock sends predefined Measurements with a 1 Second Interval and 19 | * emits them to the SourceContext. This is done, so that we can simulate Measurements and check if the Warnings 20 | * are correctly generated. 21 | * 22 | * SevereHeatWarning (WBAN = PW01, First Measurement = (Date = 2016-07-02, Time = 12:00, Temperature = 33.000000), Second Measurement = (Date = 2016-07-03, Time = 19:00, Temperature = 33.000000)) 23 | */ 24 | public class SevereHeatWarningSourceFunction extends PeriodicEmittingDataSourceFunction { 25 | 26 | @Override 27 | protected Iterable iterable() { 28 | Station station = new Station("PW01", "Philipps Station", "Unknown", "Germany", 1, new GeoLocation(52.5179604,13.3775465)); 29 | 30 | List sample = Arrays.asList( 31 | new LocalWeatherData(station, LocalDate.of(2016, 7, 1), LocalTime.of(0, 0), 15.0f, 10.0f, 0.0f, "CLR"), 32 | new LocalWeatherData(station, LocalDate.of(2016, 7, 1), LocalTime.of(1, 0), 15.0f, 10.0f, 0.0f, "CLR"), 33 | new LocalWeatherData(station, LocalDate.of(2016, 7, 1), LocalTime.of(7, 0), 15.0f, 10.0f, 0.0f, "CLR"), 34 | new LocalWeatherData(station, LocalDate.of(2016, 7, 2), LocalTime.of(12, 0), 33.0f, 10.0f, 0.0f, "CLR"), 35 | new LocalWeatherData(station, LocalDate.of(2016, 7, 2), LocalTime.of(14, 0), 15.0f, 10.0f, 0.0f, "CLR"), 36 | new LocalWeatherData(station, LocalDate.of(2016, 7, 2), LocalTime.of(16, 0), 15.0f, 10.0f, 0.0f, "CLR"), 37 | new LocalWeatherData(station, LocalDate.of(2016, 7, 3), LocalTime.of(17, 0), 15.0f, 10.0f, 0.0f, "CLR"), 38 | new LocalWeatherData(station, LocalDate.of(2016, 7, 3), LocalTime.of(18, 0), 15.0f, 10.0f, 0.0f, "CLR"), 39 | new LocalWeatherData(station, LocalDate.of(2016, 7, 3), LocalTime.of(19, 0), 33.0f, 10.0f, 0.0f, "CLR")); 40 | 41 | return sample; 42 | } 43 | 44 | @Override 45 | protected Duration interval() { 46 | return Duration.ofSeconds(1); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/IWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model; 5 | 6 | /** 7 | * Marker interface used for Warnings. 8 | */ 9 | public interface IWarning { 10 | 11 | } 12 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/IWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model; 5 | 6 | import org.apache.flink.cep.pattern.Pattern; 7 | 8 | import java.io.Serializable; 9 | import java.util.List; 10 | import java.util.Map; 11 | 12 | /** 13 | * A Warning Pattern describes the pattern of a Warning, which is triggered by an Event. 14 | * 15 | * @param Event Type 16 | * @param Warning Type 17 | */ 18 | public interface IWarningPattern extends Serializable { 19 | 20 | /** 21 | * Implements the mapping between the pattern matching result and the warning. 22 | * 23 | * @param pattern Pattern, which has been matched by Apache Flink. 24 | * @return The warning created from the given match result. 25 | */ 26 | TWarningType create(Map> pattern); 27 | 28 | /** 29 | * Implementes the Apache Flink CEP Event Pattern which triggers a warning. 30 | * 31 | * @return The Apache Flink CEP Pattern definition. 32 | */ 33 | Pattern getEventPattern(); 34 | 35 | /** 36 | * Returns the Warning Class for simplifying reflection. 37 | * 38 | * @return Class Type of the Warning. 39 | */ 40 | Class getWarningTargetType(); 41 | 42 | } 43 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/patterns/temperature/ExcessiveHeatWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.patterns.temperature; 5 | 6 | import app.cep.model.IWarningPattern; 7 | import app.cep.model.warnings.temperature.ExcessiveHeatWarning; 8 | import model.LocalWeatherData; 9 | import org.apache.flink.cep.pattern.Pattern; 10 | import org.apache.flink.cep.pattern.conditions.SimpleCondition; 11 | import org.apache.flink.streaming.api.windowing.time.Time; 12 | 13 | import java.util.List; 14 | import java.util.Map; 15 | 16 | /** 17 | * Excessive Heat Warning – Extreme Heat Index (HI) values forecast to meet or exceed locally defined warning criteria for at least two days. 18 | * Specific criteria varies among local Weather Forecast Offices, due to climate variability and the effect of excessive heat on the local 19 | * population. 20 | * 21 | * Typical HI values are maximum daytime temperatures above 105 to 110 °F (41 to 43 °C) and minimum nighttime temperatures above 75 °F (24 °C). 22 | */ 23 | public class ExcessiveHeatWarningPattern implements IWarningPattern { 24 | 25 | public ExcessiveHeatWarningPattern() {} 26 | 27 | @Override 28 | public ExcessiveHeatWarning create(Map> pattern) { 29 | LocalWeatherData first = pattern.get("First Event").get(0); 30 | LocalWeatherData second = pattern.get("Second Event").get(0); 31 | 32 | return new ExcessiveHeatWarning(first, second); 33 | } 34 | 35 | @Override 36 | public Pattern getEventPattern() { 37 | return Pattern 38 | .begin("First Event").where( 39 | new SimpleCondition() { 40 | @Override 41 | public boolean filter(LocalWeatherData event) throws Exception { 42 | return event.getTemperature() >= 41.0f; 43 | } 44 | }) 45 | .next("Second Event").where( 46 | new SimpleCondition() { 47 | @Override 48 | public boolean filter(LocalWeatherData event) throws Exception { 49 | return event.getTemperature() >= 41.0f; 50 | } 51 | }) 52 | .within(Time.days(2)); 53 | } 54 | 55 | @Override 56 | public Class getWarningTargetType() { 57 | return ExcessiveHeatWarning.class; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/patterns/temperature/ExtremeColdWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.patterns.temperature; 5 | 6 | import app.cep.model.IWarningPattern; 7 | import app.cep.model.warnings.temperature.ExtremeColdWarning; 8 | import model.LocalWeatherData; 9 | import org.apache.flink.cep.pattern.Pattern; 10 | import org.apache.flink.cep.pattern.conditions.SimpleCondition; 11 | import org.apache.flink.streaming.api.windowing.time.Time; 12 | 13 | import java.util.List; 14 | import java.util.Map; 15 | 16 | /** 17 | * Extreme Cold Warning – Forecast shelter temperature of −50 °F (−46 °C) or colder and air temperature remains below −40 °F (−40 °C) up to the 700 mb (21 inHg) level for 18 | * three or more consecutive days. Only issued by Weather Forecast Offices in Alaska. Elsewhere, an Extreme Cold Warning can be issued by local Weather Forecast Offices 19 | * as an experimental product using locally appropriate thresholds, usually with little or no wind. 20 | */ 21 | public class ExtremeColdWarningPattern implements IWarningPattern { 22 | 23 | public ExtremeColdWarningPattern() { 24 | 25 | } 26 | 27 | @Override 28 | public ExtremeColdWarning create(Map> pattern) { 29 | LocalWeatherData first = (LocalWeatherData) pattern.get("First Event").get(0); 30 | LocalWeatherData second = (LocalWeatherData) pattern.get("Second Event").get(0); 31 | LocalWeatherData third = (LocalWeatherData) pattern.get("Third Event").get(0); 32 | 33 | return new ExtremeColdWarning(first, second, third); 34 | } 35 | 36 | @Override 37 | public Pattern getEventPattern() { 38 | return Pattern 39 | .begin("First Event").where( 40 | new SimpleCondition() { 41 | @Override 42 | public boolean filter(LocalWeatherData event) throws Exception { 43 | return event.getTemperature() <= -46.0f; 44 | } 45 | }) 46 | .next("Second Event").where( 47 | new SimpleCondition() { 48 | @Override 49 | public boolean filter(LocalWeatherData event) throws Exception { 50 | return event.getTemperature() <= -46.0f; 51 | } 52 | }) 53 | .next("Third Event").where( 54 | new SimpleCondition() { 55 | @Override 56 | public boolean filter(LocalWeatherData event) throws Exception { 57 | return event.getTemperature() <= -46.0f; 58 | } 59 | }) 60 | .within(Time.days(3)); 61 | } 62 | 63 | @Override 64 | public Class getWarningTargetType() { 65 | return ExtremeColdWarning.class; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/patterns/temperature/SevereHeatWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.patterns.temperature; 5 | 6 | import app.cep.model.IWarningPattern; 7 | import app.cep.model.warnings.temperature.ExcessiveHeatWarning; 8 | import app.cep.model.warnings.temperature.SevereHeatWarning; 9 | import model.LocalWeatherData; 10 | import org.apache.flink.cep.pattern.Pattern; 11 | import org.apache.flink.cep.pattern.conditions.SimpleCondition; 12 | import org.apache.flink.streaming.api.windowing.time.Time; 13 | 14 | import java.util.List; 15 | import java.util.Map; 16 | 17 | public class SevereHeatWarningPattern implements IWarningPattern { 18 | 19 | public SevereHeatWarningPattern() {} 20 | 21 | @Override 22 | public SevereHeatWarning create(Map> pattern) { 23 | LocalWeatherData first = pattern.get("First Event").get(0); 24 | LocalWeatherData second = pattern.get("Second Event").get(0); 25 | 26 | return new SevereHeatWarning(first, second); 27 | } 28 | 29 | @Override 30 | public Pattern getEventPattern() { 31 | return Pattern 32 | .begin("First Event").where( 33 | new SimpleCondition() { 34 | @Override 35 | public boolean filter(LocalWeatherData event) throws Exception { 36 | return event.getTemperature() >= 30.0f; 37 | } 38 | }) 39 | .next("Second Event").where( 40 | new SimpleCondition() { 41 | @Override 42 | public boolean filter(LocalWeatherData event) throws Exception { 43 | return event.getTemperature() >= 30.0f; 44 | } 45 | }) 46 | .within(Time.days(2)); 47 | } 48 | 49 | @Override 50 | public Class getWarningTargetType() { 51 | return SevereHeatWarning.class; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/patterns/wind/ExtremeWindWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.patterns.wind; 5 | 6 | import app.cep.model.IWarningPattern; 7 | import app.cep.model.warnings.temperature.ExtremeColdWarning; 8 | import app.cep.model.warnings.wind.ExtremeWindWarning; 9 | import model.LocalWeatherData; 10 | import org.apache.flink.cep.pattern.Pattern; 11 | import org.apache.flink.cep.pattern.conditions.SimpleCondition; 12 | 13 | import java.util.List; 14 | import java.util.Map; 15 | 16 | public class ExtremeWindWarningPattern implements IWarningPattern { 17 | 18 | public ExtremeWindWarningPattern() { 19 | 20 | } 21 | 22 | @Override 23 | public ExtremeWindWarning create(Map> pattern) { 24 | LocalWeatherData first = pattern.get("First Event").get(0); 25 | 26 | return new ExtremeWindWarning(first); 27 | } 28 | 29 | @Override 30 | public Pattern getEventPattern() { 31 | return Pattern 32 | .begin("First Event").where( 33 | new SimpleCondition() { 34 | @Override 35 | public boolean filter(LocalWeatherData event) throws Exception { 36 | return event.getWindSpeed() > 110; 37 | } 38 | }); 39 | } 40 | 41 | @Override 42 | public Class getWarningTargetType() { 43 | return ExtremeWindWarning.class; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/patterns/wind/HighWindWarningPattern.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.patterns.wind; 5 | 6 | import app.cep.model.IWarningPattern; 7 | import app.cep.model.warnings.wind.HighWindWarning; 8 | import model.LocalWeatherData; 9 | import org.apache.flink.cep.pattern.Pattern; 10 | import org.apache.flink.cep.pattern.conditions.SimpleCondition; 11 | 12 | import java.util.List; 13 | import java.util.Map; 14 | 15 | public class HighWindWarningPattern implements IWarningPattern { 16 | 17 | public HighWindWarningPattern() { 18 | 19 | } 20 | 21 | @Override 22 | public HighWindWarning create(Map> pattern) { 23 | LocalWeatherData first = pattern.get("First Event").get(0); 24 | 25 | return new HighWindWarning(first); 26 | } 27 | 28 | @Override 29 | public Pattern getEventPattern() { 30 | return Pattern 31 | .begin("First Event").where( 32 | new SimpleCondition() { 33 | @Override 34 | public boolean filter(LocalWeatherData event) throws Exception { 35 | return event.getWindSpeed() >= 39 && event.getWindSpeed() <= 110; 36 | } 37 | }); 38 | } 39 | 40 | @Override 41 | public Class getWarningTargetType() { 42 | return HighWindWarning.class; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/warnings/temperature/ExcessiveHeatWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.warnings.temperature; 5 | 6 | import app.cep.model.IWarning; 7 | import model.LocalWeatherData; 8 | 9 | public class ExcessiveHeatWarning implements IWarning { 10 | 11 | private final LocalWeatherData localWeatherData0; 12 | private final LocalWeatherData localWeatherData1; 13 | 14 | public ExcessiveHeatWarning(LocalWeatherData localWeatherData0, LocalWeatherData localWeatherData1) { 15 | this.localWeatherData0 = localWeatherData0; 16 | this.localWeatherData1 = localWeatherData1; 17 | } 18 | 19 | public LocalWeatherData getLocalWeatherData0() { 20 | return localWeatherData0; 21 | } 22 | 23 | public LocalWeatherData getLocalWeatherData1() { 24 | return localWeatherData1; 25 | } 26 | 27 | @Override 28 | public String toString() { 29 | return String.format("ExcessiveHeatWarning (WBAN = %s, First Measurement = (%s), Second Measurement = (%s))", 30 | localWeatherData0.getStation().getWban(), 31 | getEventSummary(localWeatherData0), 32 | getEventSummary(localWeatherData1)); 33 | } 34 | 35 | private String getEventSummary(LocalWeatherData localWeatherData) { 36 | 37 | return String.format("Date = %s, Time = %s, Temperature = %f", 38 | localWeatherData.getDate(), localWeatherData.getTime(), localWeatherData.getTemperature()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/warnings/temperature/ExtremeColdWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.warnings.temperature; 5 | 6 | import app.cep.model.IWarning; 7 | import model.LocalWeatherData; 8 | import org.apache.flink.cep.pattern.Pattern; 9 | import org.apache.flink.streaming.api.windowing.time.Time; 10 | 11 | import java.util.Map; 12 | 13 | public class ExtremeColdWarning implements IWarning { 14 | 15 | private final LocalWeatherData localWeatherData0; 16 | private final LocalWeatherData localWeatherData1; 17 | private final LocalWeatherData localWeatherData2; 18 | 19 | public ExtremeColdWarning(LocalWeatherData localWeatherData0, LocalWeatherData localWeatherData1, LocalWeatherData localWeatherData2) { 20 | this.localWeatherData0 = localWeatherData0; 21 | this.localWeatherData1 = localWeatherData1; 22 | this.localWeatherData2 = localWeatherData2; 23 | } 24 | 25 | public LocalWeatherData getLocalWeatherData0() { 26 | return localWeatherData0; 27 | } 28 | 29 | public LocalWeatherData getLocalWeatherData1() { 30 | return localWeatherData1; 31 | } 32 | 33 | public LocalWeatherData getLocalWeatherData2() { 34 | return localWeatherData2; 35 | } 36 | 37 | @Override 38 | public String toString() { 39 | return String.format("ExtremeColdWarning (WBAN = %s, First Measurement = (%s), Second Measurement = (%s))", 40 | localWeatherData0.getStation().getWban(), 41 | getEventSummary(localWeatherData0), 42 | getEventSummary(localWeatherData1)); 43 | } 44 | 45 | private String getEventSummary(LocalWeatherData localWeatherData) { 46 | 47 | return String.format("Date = %s, Time = %s, Temperature = %f", 48 | localWeatherData.getDate(), localWeatherData.getTime(), localWeatherData.getTemperature()); 49 | } 50 | 51 | } 52 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/warnings/temperature/SevereHeatWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.warnings.temperature; 5 | 6 | import app.cep.model.IWarning; 7 | import model.LocalWeatherData; 8 | 9 | public class SevereHeatWarning implements IWarning { 10 | 11 | private final LocalWeatherData localWeatherData0; 12 | private final LocalWeatherData localWeatherData1; 13 | 14 | public SevereHeatWarning(LocalWeatherData localWeatherData0, LocalWeatherData localWeatherData1) { 15 | this.localWeatherData0 = localWeatherData0; 16 | this.localWeatherData1 = localWeatherData1; 17 | } 18 | 19 | public LocalWeatherData getLocalWeatherData0() { 20 | return localWeatherData0; 21 | } 22 | 23 | public LocalWeatherData getLocalWeatherData1() { 24 | return localWeatherData1; 25 | } 26 | 27 | @Override 28 | public String toString() { 29 | return String.format("SevereHeatWarning (WBAN = %s, First Measurement = (%s), Second Measurement = (%s))", 30 | localWeatherData0.getStation().getWban(), 31 | getEventSummary(localWeatherData0), 32 | getEventSummary(localWeatherData1)); 33 | } 34 | 35 | private String getEventSummary(LocalWeatherData localWeatherData) { 36 | 37 | return String.format("Date = %s, Time = %s, Temperature = %f", 38 | localWeatherData.getDate(), localWeatherData.getTime(), localWeatherData.getTemperature()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/warnings/wind/ExtremeWindWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.warnings.wind; 5 | 6 | import app.cep.model.IWarning; 7 | import model.LocalWeatherData; 8 | 9 | public class ExtremeWindWarning implements IWarning { 10 | 11 | private final LocalWeatherData localWeatherData; 12 | 13 | public ExtremeWindWarning(LocalWeatherData localWeatherData) { 14 | this.localWeatherData = localWeatherData; 15 | } 16 | 17 | public LocalWeatherData getLocalWeatherData() { 18 | return localWeatherData; 19 | } 20 | 21 | @Override 22 | public String toString() { 23 | return String.format("ExtremeWindWarning (WBAN = %s, First Measurement = (%s))", 24 | localWeatherData.getStation().getWban(), 25 | getEventSummary(localWeatherData)); 26 | } 27 | 28 | private String getEventSummary(LocalWeatherData localWeatherData) { 29 | 30 | return String.format("Date = %s, Time = %s, WindSpeed = %f", 31 | localWeatherData.getDate(), localWeatherData.getTime(), localWeatherData.getWindSpeed()); 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/app/cep/model/warnings/wind/HighWindWarning.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package app.cep.model.warnings.wind; 5 | 6 | import app.cep.model.IWarning; 7 | import model.LocalWeatherData; 8 | 9 | public class HighWindWarning implements IWarning { 10 | 11 | private final LocalWeatherData localWeatherData; 12 | 13 | public HighWindWarning(LocalWeatherData localWeatherData) { 14 | this.localWeatherData = localWeatherData; 15 | } 16 | 17 | public LocalWeatherData getLocalWeatherData() { 18 | return localWeatherData; 19 | } 20 | 21 | @Override 22 | public String toString() { 23 | return String.format("HighWindWarning (WBAN = %s, First Measurement = (%s))", 24 | localWeatherData.getStation().getWban(), 25 | getEventSummary(localWeatherData)); 26 | } 27 | 28 | private String getEventSummary(LocalWeatherData localWeatherData) { 29 | 30 | return String.format("Date = %s, Time = %s, WindSpeed = %f", 31 | localWeatherData.getDate(), localWeatherData.getTime(), localWeatherData.getWindSpeed()); 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/mapping/LocalWeatherDataMapper.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.mapping; 5 | 6 | import csv.model.LocalWeatherData; 7 | import de.bytefish.jtinycsvparser.builder.IObjectCreator; 8 | import de.bytefish.jtinycsvparser.mapping.CsvMapping; 9 | import de.bytefish.jtinycsvparser.typeconverter.FloatConverter; 10 | import de.bytefish.jtinycsvparser.typeconverter.IgnoreMissingValuesConverter; 11 | import de.bytefish.jtinycsvparser.typeconverter.LocalDateConverter; 12 | import de.bytefish.jtinycsvparser.typeconverter.LocalTimeConverter; 13 | 14 | import java.time.LocalDate; 15 | import java.time.LocalTime; 16 | import java.time.format.DateTimeFormatter; 17 | 18 | public class LocalWeatherDataMapper extends CsvMapping 19 | { 20 | public LocalWeatherDataMapper(IObjectCreator creator) 21 | { 22 | super(creator); 23 | 24 | mapProperty(0, String.class, LocalWeatherData::setWban); 25 | mapProperty(1, LocalDate.class, LocalWeatherData::setDate, new LocalDateConverter(DateTimeFormatter.ofPattern("yyyyMMdd"))); 26 | mapProperty(2, LocalTime.class, LocalWeatherData::setTime, new LocalTimeConverter(DateTimeFormatter.ofPattern("HHmm"))); 27 | mapProperty(4, String.class, LocalWeatherData::setSkyCondition); 28 | mapProperty(12, Float.class, LocalWeatherData::setDryBulbCelsius, new IgnoreMissingValuesConverter(new FloatConverter(), "M")); 29 | mapProperty(24, Float.class, LocalWeatherData::setWindSpeed, new IgnoreMissingValuesConverter(new FloatConverter(), "M")); 30 | mapProperty(30, Float.class, LocalWeatherData::setStationPressure, new IgnoreMissingValuesConverter(new FloatConverter(), "M")); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/mapping/StationMapper.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.mapping; 5 | 6 | import csv.model.Station; 7 | import de.bytefish.jtinycsvparser.builder.IObjectCreator; 8 | import de.bytefish.jtinycsvparser.mapping.CsvMapping; 9 | import de.bytefish.jtinycsvparser.typeconverter.IgnoreMissingValuesConverter; 10 | import de.bytefish.jtinycsvparser.typeconverter.IntegerConverter; 11 | 12 | public class StationMapper extends CsvMapping 13 | { 14 | public StationMapper(IObjectCreator creator) 15 | { 16 | super(creator); 17 | 18 | mapProperty(0, String.class, Station::setWban); 19 | mapProperty(1, String.class, Station::setWmo); 20 | mapProperty(2, String.class, Station::setCallSign); 21 | mapProperty(3, String.class, Station::setClimateDivisionCode); 22 | mapProperty(4, String.class, Station::setClimateDivisionStateCode); 23 | mapProperty(5, String.class, Station::setClimateDivisionStationCode); 24 | mapProperty(6, String.class, Station::setName); 25 | mapProperty(7, String.class, Station::setState); 26 | mapProperty(8, String.class, Station::setLocation); 27 | mapProperty(9, Float.class, Station::setLatitude); 28 | mapProperty(10, Float.class, Station::setLongitude); 29 | mapProperty(11, Integer.class, Station::setGroundHeight, new IgnoreMissingValuesConverter<>(new IntegerConverter())); 30 | mapProperty(12, Integer.class, Station::setStationHeight, new IgnoreMissingValuesConverter<>(new IntegerConverter())); 31 | mapProperty(13, Integer.class, Station::setBarometer, new IgnoreMissingValuesConverter<>(new IntegerConverter())); 32 | mapProperty(14, Integer.class, Station::setTimeZone); 33 | } 34 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/model/LocalWeatherData.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.model; 5 | 6 | import java.time.LocalDate; 7 | import java.time.LocalTime; 8 | 9 | public class LocalWeatherData { 10 | 11 | private String wban; 12 | 13 | private LocalDate date; 14 | 15 | private LocalTime time; 16 | 17 | private String skyCondition; 18 | 19 | private Float dryBulbCelsius; 20 | 21 | private Float windSpeed; 22 | 23 | private Float stationPressure; 24 | 25 | public LocalWeatherData() { 26 | 27 | } 28 | 29 | public String getWban() { 30 | return wban; 31 | } 32 | 33 | public void setWban(String wban) { 34 | this.wban = wban; 35 | } 36 | 37 | public LocalDate getDate() { 38 | return date; 39 | } 40 | 41 | public void setDate(LocalDate date) { 42 | this.date = date; 43 | } 44 | 45 | public LocalTime getTime() { 46 | return time; 47 | } 48 | 49 | public void setTime(LocalTime time) { 50 | this.time = time; 51 | } 52 | 53 | public String getSkyCondition() { 54 | return skyCondition; 55 | } 56 | 57 | public void setSkyCondition(String skyCondition) { 58 | this.skyCondition = skyCondition; 59 | } 60 | 61 | public Float getDryBulbCelsius() { 62 | return dryBulbCelsius; 63 | } 64 | 65 | public void setDryBulbCelsius(Float dryBulbCelsius) { 66 | this.dryBulbCelsius = dryBulbCelsius; 67 | } 68 | 69 | public Float getWindSpeed() { 70 | return windSpeed; 71 | } 72 | 73 | public void setWindSpeed(Float windSpeed) { 74 | this.windSpeed = windSpeed; 75 | } 76 | 77 | public Float getStationPressure() { 78 | return stationPressure; 79 | } 80 | 81 | public void setStationPressure(Float stationPressure) { 82 | this.stationPressure = stationPressure; 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/model/Station.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.model; 5 | 6 | public class Station { 7 | 8 | private String wban; 9 | 10 | private String wmo; 11 | 12 | private String callSign; 13 | 14 | private String climateDivisionCode; 15 | 16 | private String climateDivisionStateCode; 17 | 18 | private String climateDivisionStationCode; 19 | 20 | private String name; 21 | 22 | private String state; 23 | 24 | private String location; 25 | 26 | private Float latitude; 27 | 28 | private Float longitude; 29 | 30 | private Integer groundHeight; 31 | 32 | private Integer stationHeight; 33 | 34 | private Integer barometer; 35 | 36 | private Integer timeZone; 37 | 38 | public Station() { 39 | } 40 | 41 | public String getWban() { 42 | return wban; 43 | } 44 | 45 | public void setWban(String wban) { 46 | this.wban = wban; 47 | } 48 | 49 | public String getWmo() { 50 | return wmo; 51 | } 52 | 53 | public void setWmo(String wmo) { 54 | this.wmo = wmo; 55 | } 56 | 57 | public String getCallSign() { 58 | return callSign; 59 | } 60 | 61 | public void setCallSign(String callSign) { 62 | this.callSign = callSign; 63 | } 64 | 65 | public String getClimateDivisionCode() { 66 | return climateDivisionCode; 67 | } 68 | 69 | public void setClimateDivisionCode(String climateDivisionCode) { 70 | this.climateDivisionCode = climateDivisionCode; 71 | } 72 | 73 | public String getClimateDivisionStateCode() { 74 | return climateDivisionStateCode; 75 | } 76 | 77 | public void setClimateDivisionStateCode(String climateDivisionStateCode) { 78 | this.climateDivisionStateCode = climateDivisionStateCode; 79 | } 80 | 81 | public String getClimateDivisionStationCode() { 82 | return climateDivisionStationCode; 83 | } 84 | 85 | public void setClimateDivisionStationCode(String climateDivisionStationCode) { 86 | this.climateDivisionStationCode = climateDivisionStationCode; 87 | } 88 | 89 | public String getName() { 90 | return name; 91 | } 92 | 93 | public void setName(String name) { 94 | this.name = name; 95 | } 96 | 97 | public String getState() { 98 | return state; 99 | } 100 | 101 | public void setState(String state) { 102 | this.state = state; 103 | } 104 | 105 | public String getLocation() { 106 | return location; 107 | } 108 | 109 | public void setLocation(String location) { 110 | this.location = location; 111 | } 112 | 113 | public float getLatitude() { 114 | return latitude; 115 | } 116 | 117 | public void setLatitude(Float latitude) { 118 | this.latitude = latitude; 119 | } 120 | 121 | public float getLongitude() { 122 | return longitude; 123 | } 124 | 125 | public void setLongitude(Float longitude) { 126 | this.longitude = longitude; 127 | } 128 | 129 | public Integer getGroundHeight() { 130 | return groundHeight; 131 | } 132 | 133 | public void setGroundHeight(Integer groundHeight) { 134 | this.groundHeight = groundHeight; 135 | } 136 | 137 | public Integer getStationHeight() { 138 | return stationHeight; 139 | } 140 | 141 | public void setStationHeight(Integer stationHeight) { 142 | this.stationHeight = stationHeight; 143 | } 144 | 145 | public Integer getBarometer() { 146 | return barometer; 147 | } 148 | 149 | public void setBarometer(Integer barometer) { 150 | this.barometer = barometer; 151 | } 152 | 153 | public Integer getTimeZone() { 154 | return timeZone; 155 | } 156 | 157 | public void setTimeZone(Integer timeZone) { 158 | this.timeZone = timeZone; 159 | } 160 | } 161 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/parser/Parsers.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.parser; 5 | 6 | import csv.mapping.LocalWeatherDataMapper; 7 | import csv.mapping.StationMapper; 8 | import csv.model.LocalWeatherData; 9 | import csv.model.Station; 10 | import de.bytefish.jtinycsvparser.CsvParser; 11 | import de.bytefish.jtinycsvparser.CsvParserOptions; 12 | import de.bytefish.jtinycsvparser.tokenizer.StringSplitTokenizer; 13 | 14 | public class Parsers { 15 | 16 | public static CsvParser StationParser() { 17 | 18 | return new CsvParser<>(new CsvParserOptions(true, new StringSplitTokenizer("\\|", true)), new StationMapper(() -> new Station())); 19 | } 20 | 21 | public static CsvParser LocalWeatherDataParser() 22 | { 23 | return new CsvParser<>(new CsvParserOptions(true, new StringSplitTokenizer(",", true)), new LocalWeatherDataMapper(() -> new LocalWeatherData())); 24 | } 25 | 26 | } 27 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/csv/sorting/PrepareWeatherData.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package csv.sorting; 5 | 6 | import csv.model.Station; 7 | import csv.parser.Parsers; 8 | import de.bytefish.jtinycsvparser.mapping.CsvMappingResult; 9 | import org.apache.commons.lang3.tuple.ImmutablePair; 10 | 11 | import java.io.BufferedWriter; 12 | import java.nio.charset.StandardCharsets; 13 | import java.nio.file.FileSystems; 14 | import java.nio.file.Files; 15 | import java.nio.file.Path; 16 | import java.time.OffsetDateTime; 17 | import java.time.ZoneOffset; 18 | import java.util.ArrayList; 19 | import java.util.Comparator; 20 | import java.util.List; 21 | import java.util.Map; 22 | import java.util.concurrent.atomic.AtomicInteger; 23 | import java.util.stream.Collectors; 24 | import java.util.stream.Stream; 25 | 26 | public class PrepareWeatherData { 27 | 28 | public static void main(String[] args) throws Exception { 29 | 30 | // Path to read the CSV data from: 31 | final Path csvStationDataFilePath = FileSystems.getDefault().getPath("D:\\datasets\\201503station.txt"); 32 | final Path csvLocalWeatherDataUnsortedFilePath = FileSystems.getDefault().getPath("D:\\Datasets\\201503hourly.txt"); 33 | final Path csvLocalWeatherDataSortedFilePath = FileSystems.getDefault().getPath("D:\\Datasets\\201503hourly_sorted.txt"); 34 | 35 | // A map between the WBAN and Station for faster Lookups: 36 | final Map stationMap = getStationMap(csvStationDataFilePath); 37 | 38 | // Holds the List of Sorted DateTimes (including ZoneOffset): 39 | List indices = new ArrayList<>(); 40 | 41 | // Comparator for sorting the File: 42 | Comparator byMeasurementTime = (e1, e2) -> e1.compareTo(e2); 43 | 44 | // Get the sorted indices from the stream of LocalWeatherData Elements: 45 | try (Stream> stream = getLocalWeatherData(csvLocalWeatherDataUnsortedFilePath)) { 46 | 47 | // Holds the current line index, when processing the input Stream: 48 | AtomicInteger currentIndex = new AtomicInteger(1); 49 | 50 | // We want to get a list of indices, which sorts the CSV file by measurement time: 51 | indices = stream 52 | // Skip the CSV Header: 53 | .skip(1) 54 | // Start by enumerating ALL mapping results: 55 | .map(x -> new ImmutablePair<>(currentIndex.getAndAdd(1), x)) 56 | // Then only take those lines, that are actually valid: 57 | .filter(x -> x.getRight().isValid()) 58 | // Now take the parsed entity from the CsvMappingResult: 59 | .map(x -> new ImmutablePair<>(x.getLeft(), x.getRight().getResult())) 60 | // Take only those measurements, that are also available in the list of stations: 61 | .filter(x -> stationMap.containsKey(x.getRight().getWban())) 62 | // Get the OffsetDateTime from the LocalWeatherData, which includes the ZoneOffset of the Station: 63 | .map(x -> { 64 | // Get the matching station: 65 | csv.model.Station station = stationMap.get(x.getRight().getWban()); 66 | // Calculate the OffsetDateTime from the given measurement: 67 | OffsetDateTime measurementTime = OffsetDateTime.of(x.getRight().getDate(), x.getRight().getTime(), ZoneOffset.ofHours(0)); 68 | // Build the Immutable pair with the Index again: 69 | return new ImmutablePair<>(x.getLeft(), measurementTime); 70 | }) 71 | // Now sort the Measurements by their Timestamp: 72 | .sorted((x, y) -> byMeasurementTime.compare(x.getRight(), y.getRight())) 73 | // Take only the Index: 74 | .map(x -> x.getLeft()) 75 | // And turn it into a List: 76 | .collect(Collectors.toList()); 77 | } 78 | 79 | // Now sorts the File by Line Number: 80 | writeSortedFileByIndices(csvLocalWeatherDataUnsortedFilePath, indices, csvLocalWeatherDataSortedFilePath); 81 | } 82 | 83 | 84 | 85 | private static void writeSortedFileByIndices(Path csvFileIn, List indices, Path csvFileOut) { 86 | try { 87 | List csvDataList = new ArrayList<>(); 88 | 89 | // This is sorting for the dumb (like me). Read the entire CSV file, skipping the first line: 90 | try (Stream lines = Files.lines(csvFileIn, StandardCharsets.US_ASCII).skip(1)) 91 | { 92 | csvDataList = lines.collect(Collectors.toList()); 93 | } 94 | // Now write the sorted file: 95 | try(BufferedWriter writer = Files.newBufferedWriter(csvFileOut)) { 96 | for (Integer index : indices) { 97 | writer.write(csvDataList.get(index)); 98 | writer.newLine(); 99 | } 100 | } 101 | } catch(Exception e) { 102 | throw new RuntimeException(e); 103 | } 104 | } 105 | 106 | private static Stream> getLocalWeatherData(Path path) { 107 | return Parsers.LocalWeatherDataParser().readFromFile(path, StandardCharsets.US_ASCII); 108 | } 109 | 110 | private static Stream getStations(Path path) { 111 | return Parsers.StationParser().readFromFile(path, StandardCharsets.US_ASCII) 112 | .filter(x -> x.isValid()) 113 | .map(x -> x.getResult()); 114 | } 115 | 116 | private static Map getStationMap(Path path) { 117 | try (Stream stationStream = getStations(path)) { 118 | return stationStream 119 | // Group by WBAN, there are probably faulty entries with duplicate Station IDs: 120 | .collect(Collectors.groupingBy(Station::getWban)) 121 | // Turn into a Stream again: 122 | .entrySet().stream() 123 | // Make a best guess and take the first one: 124 | .collect(Collectors.toMap(x -> x.getKey(), x -> x.getValue().get(0))); 125 | } 126 | } 127 | } 128 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/elastic/converter/LocalWeatherDataConverter.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package elastic.converter; 5 | 6 | import utils.DateUtilities; 7 | 8 | import java.time.ZoneOffset; 9 | 10 | public class LocalWeatherDataConverter { 11 | 12 | public static elastic.model.LocalWeatherData convert(model.LocalWeatherData modelLocalWeatherData) { 13 | 14 | elastic.model.LocalWeatherData elasticLocalWeatherData = new elastic.model.LocalWeatherData(); 15 | 16 | elasticLocalWeatherData.dateTime = DateUtilities.from(modelLocalWeatherData.getDate(), modelLocalWeatherData.getTime(), ZoneOffset.ofHours(modelLocalWeatherData.getStation().getTimeZone())); 17 | elasticLocalWeatherData.skyCondition = modelLocalWeatherData.getSkyCondition(); 18 | elasticLocalWeatherData.stationPressure = modelLocalWeatherData.getStationPressure(); 19 | elasticLocalWeatherData.temperature = modelLocalWeatherData.getTemperature(); 20 | elasticLocalWeatherData.windSpeed = modelLocalWeatherData.getWindSpeed(); 21 | 22 | // Convert the Station data: 23 | elasticLocalWeatherData.station = convert(modelLocalWeatherData.getStation()); 24 | 25 | return elasticLocalWeatherData; 26 | } 27 | 28 | private static elastic.model.Station convert(model.Station modelStation) { 29 | elastic.model.Station elasticStation = new elastic.model.Station(); 30 | 31 | elasticStation.wban = modelStation.getWban(); 32 | elasticStation.name = modelStation.getName(); 33 | elasticStation.state = modelStation.getState(); 34 | elasticStation.location = modelStation.getLocation(); 35 | 36 | // Convert the GeoLocation: 37 | elasticStation.geoLocation = convert(modelStation.getGeoLocation()); 38 | 39 | return elasticStation; 40 | } 41 | 42 | private static elastic.model.GeoLocation convert(model.GeoLocation modelGeoLocation) { 43 | elastic.model.GeoLocation elasticGeoLocation = new elastic.model.GeoLocation(); 44 | 45 | elasticGeoLocation.lat = modelGeoLocation.getLat(); 46 | elasticGeoLocation.lon = modelGeoLocation.getLon(); 47 | 48 | return elasticGeoLocation; 49 | } 50 | 51 | } 52 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/elastic/mapping/LocalWeatherDataMapper.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package elastic.mapping; 5 | 6 | 7 | import de.bytefish.elasticutils.elasticsearch6.mapping.BaseElasticSearchMapping; 8 | import org.elasticsearch.Version; 9 | import org.elasticsearch.index.mapper.RootObjectMapper; 10 | import org.elasticsearch.index.mapper.DateFieldMapper; 11 | import org.elasticsearch.index.mapper.TextFieldMapper; 12 | import org.elasticsearch.index.mapper.ObjectMapper; 13 | import org.elasticsearch.index.mapper.GeoPointFieldMapper; 14 | import org.elasticsearch.index.mapper.NumberFieldMapper; 15 | import org.elasticsearch.index.mapper.NumberFieldMapper.NumberType; 16 | 17 | public class LocalWeatherDataMapper extends BaseElasticSearchMapping { 18 | 19 | private static final String INDEX_TYPE = "document"; 20 | 21 | public LocalWeatherDataMapper() { 22 | super(INDEX_TYPE, Version.V_6_5_1); 23 | } 24 | 25 | @Override 26 | protected void configureRootObjectBuilder(RootObjectMapper.Builder builder) { 27 | builder 28 | .add(new DateFieldMapper.Builder("dateTime")) 29 | .add(new NumberFieldMapper.Builder("temperature", NumberType.FLOAT)) 30 | .add(new NumberFieldMapper.Builder("windSpeed", NumberType.FLOAT)) 31 | .add(new NumberFieldMapper.Builder("stationPressure", NumberType.FLOAT)) 32 | .add(new TextFieldMapper.Builder("skyCondition")) 33 | .add(new ObjectMapper.Builder("station") 34 | .add(new TextFieldMapper.Builder("wban")) 35 | .add(new TextFieldMapper.Builder("name")) 36 | .add(new TextFieldMapper.Builder("state")) 37 | .add(new TextFieldMapper.Builder("location")) 38 | .add(new GeoPointFieldMapper.Builder("coordinates")) 39 | .nested(ObjectMapper.Nested.newNested(true, false))); 40 | } 41 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/elastic/model/GeoLocation.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package elastic.model; 5 | 6 | import com.fasterxml.jackson.annotation.JsonProperty; 7 | 8 | public class GeoLocation { 9 | 10 | @JsonProperty("lat") 11 | public double lat; 12 | 13 | @JsonProperty("lon") 14 | public double lon; 15 | 16 | public GeoLocation() {} 17 | 18 | public GeoLocation(double lat, double lon) { 19 | this.lat = lat; 20 | this.lon = lon; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/elastic/model/LocalWeatherData.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package elastic.model; 5 | 6 | import com.fasterxml.jackson.annotation.JsonProperty; 7 | 8 | import java.util.Date; 9 | 10 | public class LocalWeatherData { 11 | 12 | @JsonProperty("station") 13 | public Station station; 14 | 15 | @JsonProperty("dateTime") 16 | public Date dateTime; 17 | 18 | @JsonProperty("temperature") 19 | public Float temperature; 20 | 21 | @JsonProperty("windSpeed") 22 | public Float windSpeed; 23 | 24 | @JsonProperty("stationPressure") 25 | public Float stationPressure; 26 | 27 | @JsonProperty("skyCondition") 28 | public String skyCondition; 29 | 30 | } 31 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/elastic/model/Station.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package elastic.model; 5 | 6 | import com.fasterxml.jackson.annotation.JsonProperty; 7 | 8 | public class Station { 9 | 10 | @JsonProperty("wban") 11 | public String wban; 12 | 13 | @JsonProperty("name") 14 | public String name; 15 | 16 | @JsonProperty("state") 17 | public String state; 18 | 19 | @JsonProperty("location") 20 | public String location; 21 | 22 | @JsonProperty("coordinates") 23 | public GeoLocation geoLocation; 24 | 25 | } 26 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/model/GeoLocation.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package model; 5 | 6 | public class GeoLocation { 7 | 8 | private double lat; 9 | 10 | private double lon; 11 | 12 | public GeoLocation() { 13 | 14 | } 15 | 16 | public GeoLocation(double lat, double lon) { 17 | this.lat = lat; 18 | this.lon = lon; 19 | } 20 | 21 | public void setLat(double lat) { 22 | this.lat = lat; 23 | } 24 | 25 | public void setLon(double lon) { 26 | this.lon = lon; 27 | } 28 | 29 | public double getLat() { 30 | return lat; 31 | } 32 | 33 | public double getLon() { 34 | return lon; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/model/LocalWeatherData.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package model; 5 | 6 | import java.time.LocalDate; 7 | import java.time.LocalTime; 8 | 9 | public class LocalWeatherData { 10 | 11 | private Station station; 12 | 13 | private LocalDate date; 14 | 15 | private LocalTime time; 16 | 17 | private Float temperature; 18 | 19 | private Float windSpeed; 20 | 21 | private Float stationPressure; 22 | 23 | private String skyCondition; 24 | 25 | public LocalWeatherData() { 26 | 27 | } 28 | 29 | public LocalWeatherData(Station station, LocalDate date, LocalTime time, Float temperature, Float windSpeed, Float stationPressure, String skyCondition) { 30 | this.station = station; 31 | this.date = date; 32 | this.time = time; 33 | this.temperature = temperature; 34 | this.windSpeed = windSpeed; 35 | this.stationPressure = stationPressure; 36 | this.skyCondition = skyCondition; 37 | } 38 | 39 | public void setStation(Station station) { 40 | this.station = station; 41 | } 42 | 43 | public void setDate(LocalDate date) { 44 | this.date = date; 45 | } 46 | 47 | public void setTime(LocalTime time) { 48 | this.time = time; 49 | } 50 | 51 | public void setTemperature(Float temperature) { 52 | this.temperature = temperature; 53 | } 54 | 55 | public void setWindSpeed(Float windSpeed) { 56 | this.windSpeed = windSpeed; 57 | } 58 | 59 | public void setStationPressure(Float stationPressure) { 60 | this.stationPressure = stationPressure; 61 | } 62 | 63 | public void setSkyCondition(String skyCondition) { 64 | this.skyCondition = skyCondition; 65 | } 66 | 67 | public Station getStation() { 68 | return station; 69 | } 70 | 71 | public LocalDate getDate() { 72 | return date; 73 | } 74 | 75 | public LocalTime getTime() { 76 | return time; 77 | } 78 | 79 | public Float getTemperature() { 80 | return temperature; 81 | } 82 | 83 | public Float getWindSpeed() { 84 | return windSpeed; 85 | } 86 | 87 | public Float getStationPressure() { 88 | return stationPressure; 89 | } 90 | 91 | public String getSkyCondition() { 92 | return skyCondition; 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/model/Station.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package model; 5 | 6 | public class Station { 7 | 8 | private String wban; 9 | 10 | private String name; 11 | 12 | private String state; 13 | 14 | private String location; 15 | 16 | private Integer timeZone; 17 | 18 | private GeoLocation geoLocation; 19 | 20 | public Station() { 21 | 22 | } 23 | 24 | public Station(String wban, String name, String state, String location, Integer timeZone, GeoLocation geoLocation) { 25 | this.wban = wban; 26 | this.name = name; 27 | this.state = state; 28 | this.location = location; 29 | this.timeZone = timeZone; 30 | this.geoLocation = geoLocation; 31 | } 32 | 33 | public void setWban(String wban) { 34 | this.wban = wban; 35 | } 36 | 37 | public void setName(String name) { 38 | this.name = name; 39 | } 40 | 41 | public void setState(String state) { 42 | this.state = state; 43 | } 44 | 45 | public void setLocation(String location) { 46 | this.location = location; 47 | } 48 | 49 | public void setTimeZone(Integer timeZone) { 50 | this.timeZone = timeZone; 51 | } 52 | 53 | public void setGeoLocation(GeoLocation geoLocation) { 54 | this.geoLocation = geoLocation; 55 | } 56 | 57 | public String getWban() { 58 | return wban; 59 | } 60 | 61 | public String getName() { 62 | return name; 63 | } 64 | 65 | public String getState() { 66 | return state; 67 | } 68 | 69 | public String getLocation() { 70 | return location; 71 | } 72 | 73 | public Integer getTimeZone() { 74 | return timeZone; 75 | } 76 | 77 | public GeoLocation getGeoLocation() { 78 | return geoLocation; 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/pgsql/connection/PooledConnectionFactory.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package pgsql.connection; 5 | 6 | import org.apache.commons.dbcp2.BasicDataSource; 7 | 8 | import java.net.URI; 9 | import java.sql.Connection; 10 | import java.util.function.Supplier; 11 | 12 | public class PooledConnectionFactory implements Supplier { 13 | 14 | private final BasicDataSource connectionPool; 15 | 16 | public PooledConnectionFactory(URI databaseUri) { 17 | this.connectionPool = new BasicDataSource(); 18 | 19 | initializeConnectionPool(connectionPool, databaseUri); 20 | } 21 | 22 | private void initializeConnectionPool(BasicDataSource connectionPool, URI databaseUri) { 23 | final String dbUrl = "jdbc:postgresql://" + databaseUri.getHost() + databaseUri.getPath(); 24 | 25 | if (databaseUri.getUserInfo() != null) { 26 | connectionPool.setUsername(databaseUri.getUserInfo().split(":")[0]); 27 | connectionPool.setPassword(databaseUri.getUserInfo().split(":")[1]); 28 | } 29 | connectionPool.setDriverClassName("org.postgresql.Driver"); 30 | connectionPool.setUrl(dbUrl); 31 | connectionPool.setInitialSize(1); 32 | } 33 | 34 | @Override 35 | public Connection get() { 36 | try { 37 | return connectionPool.getConnection(); 38 | } catch(Exception e) { 39 | throw new RuntimeException(e); 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/pgsql/converter/LocalWeatherDataConverter.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package pgsql.converter; 5 | 6 | import java.time.LocalDateTime; 7 | 8 | public class LocalWeatherDataConverter { 9 | 10 | public static pgsql.model.LocalWeatherData convert(model.LocalWeatherData modelLocalWeatherData) { 11 | 12 | String wban = modelLocalWeatherData.getStation().getWban(); 13 | LocalDateTime dateTime = modelLocalWeatherData.getDate().atTime(modelLocalWeatherData.getTime()); 14 | Float temperature = modelLocalWeatherData.getTemperature(); 15 | Float windSpeed = modelLocalWeatherData.getWindSpeed(); 16 | Float stationPressure = modelLocalWeatherData.getStationPressure(); 17 | String skyCondition = modelLocalWeatherData.getSkyCondition(); 18 | 19 | return new pgsql.model.LocalWeatherData(wban, dateTime, temperature, windSpeed, stationPressure, skyCondition); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/pgsql/mapping/LocalWeatherDataMapping.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package pgsql.mapping; 5 | 6 | import de.bytefish.pgbulkinsert.mapping.AbstractMapping; 7 | import pgsql.model.LocalWeatherData; 8 | 9 | public class LocalWeatherDataMapping extends AbstractMapping { 10 | 11 | public LocalWeatherDataMapping(String schemaName, String tableName) { 12 | 13 | super(schemaName, tableName); 14 | 15 | mapVarChar("wban", pgsql.model.LocalWeatherData::getWban); 16 | mapTimeStamp("dateTime", pgsql.model.LocalWeatherData::getDateTime); 17 | mapFloat("temperature", pgsql.model.LocalWeatherData::getTemperature); 18 | mapFloat("windSpeed", pgsql.model.LocalWeatherData::getWindSpeed); 19 | mapFloat("stationPressure", pgsql.model.LocalWeatherData::getStationPressure); 20 | mapVarChar("skyCondition", pgsql.model.LocalWeatherData::getSkyCondition); 21 | } 22 | 23 | } 24 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/pgsql/model/LocalWeatherData.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package pgsql.model; 5 | 6 | import java.time.LocalDateTime; 7 | 8 | public class LocalWeatherData { 9 | 10 | private String wban; 11 | 12 | private LocalDateTime dateTime; 13 | 14 | private Float temperature; 15 | 16 | private Float windSpeed; 17 | 18 | private Float stationPressure; 19 | 20 | private String skyCondition; 21 | 22 | public LocalWeatherData(String wban, LocalDateTime dateTime, Float temperature, Float windSpeed, Float stationPressure, String skyCondition) { 23 | this.wban = wban; 24 | this.dateTime = dateTime; 25 | this.temperature = temperature; 26 | this.windSpeed = windSpeed; 27 | this.stationPressure = stationPressure; 28 | this.skyCondition = skyCondition; 29 | } 30 | 31 | public String getWban() { 32 | return wban; 33 | } 34 | 35 | public LocalDateTime getDateTime() { 36 | return dateTime; 37 | } 38 | 39 | public Float getTemperature() { 40 | return temperature; 41 | } 42 | 43 | public Float getWindSpeed() { 44 | return windSpeed; 45 | } 46 | 47 | public Float getStationPressure() { 48 | return stationPressure; 49 | } 50 | 51 | public String getSkyCondition() { 52 | return skyCondition; 53 | } 54 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sinks/elastic/BaseElasticSearchSink.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sinks.elastic; 5 | 6 | import de.bytefish.elasticutils.client.IElasticSearchClient; 7 | import de.bytefish.elasticutils.elasticsearch6.client.ElasticSearchClient; 8 | import de.bytefish.elasticutils.elasticsearch6.client.bulk.configuration.BulkProcessorConfiguration; 9 | import de.bytefish.elasticutils.elasticsearch6.client.bulk.options.BulkProcessingOptions; 10 | import de.bytefish.elasticutils.elasticsearch6.mapping.IElasticSearchMapping; 11 | import de.bytefish.elasticutils.elasticsearch6.utils.ElasticSearchUtils; 12 | import elastic.mapping.LocalWeatherDataMapper; 13 | import org.apache.flink.configuration.Configuration; 14 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 15 | import org.apache.flink.streaming.api.functions.sink.SinkFunction; 16 | import org.elasticsearch.ResourceAlreadyExistsException; 17 | import org.elasticsearch.client.Client; 18 | import org.elasticsearch.client.transport.TransportClient; 19 | import org.elasticsearch.cluster.node.DiscoveryNode; 20 | import org.elasticsearch.common.settings.Settings; 21 | import org.elasticsearch.common.transport.TransportAddress; 22 | import org.elasticsearch.transport.client.PreBuiltTransportClient; 23 | 24 | import java.net.InetAddress; 25 | import java.util.List; 26 | import java.util.concurrent.TimeUnit; 27 | 28 | public abstract class BaseElasticSearchSink extends RichSinkFunction { 29 | 30 | private final String host; 31 | private final int port; 32 | private final int bulkSize; 33 | 34 | private IElasticSearchClient client; 35 | 36 | public BaseElasticSearchSink(String host, int port, int bulkSize) { 37 | this.host = host; 38 | this.port = port; 39 | this.bulkSize = bulkSize; 40 | 41 | this.client = null; 42 | } 43 | 44 | @Override 45 | public void invoke(TEntity entity, SinkFunction.Context context) { 46 | client.index(entity); 47 | } 48 | 49 | @Override 50 | public void open(Configuration parameters) throws Exception { 51 | 52 | // According to [1] we should initialize the TransportClient **before** any Netty-related code is run. 53 | // But I simply cannot guarantee it in the current implementation, so I need to resort to a Workaround 54 | // setting the System Properties, which doesn't feel right. 55 | // 56 | // [1]: https://discuss.elastic.co/t/getting-availableprocessors-is-already-set-to-1-rejecting-1-illegalstateexception-exception/103082 57 | // 58 | System.setProperty("es.set.netty.runtime.available.processors", "false"); 59 | 60 | // Create the Transport Client: 61 | TransportClient transportClient = createClient(); 62 | 63 | // Create Index: 64 | createIndexAndMapping(transportClient); 65 | 66 | // Finally create the Client: 67 | BulkProcessingOptions options = BulkProcessingOptions.builder() 68 | .setBulkActions(bulkSize) 69 | .build(); 70 | 71 | client = new ElasticSearchClient<>(transportClient, getIndexName(), new LocalWeatherDataMapper(), new BulkProcessorConfiguration(options)); 72 | } 73 | 74 | @Override 75 | public void close() throws Exception { 76 | client.awaitClose(10, TimeUnit.SECONDS); 77 | } 78 | 79 | protected abstract String getIndexName(); 80 | 81 | protected abstract IElasticSearchMapping getMapping(); 82 | 83 | private TransportClient createClient() throws Exception { 84 | 85 | // Create a new Connection: 86 | TransportClient client = new PreBuiltTransportClient(Settings.EMPTY); 87 | 88 | client.addTransportAddress(new TransportAddress(InetAddress.getByName(host), port)); 89 | 90 | // Ensure we have connected nodes: 91 | List nodes = client.connectedNodes(); 92 | 93 | if (nodes.isEmpty()) { 94 | throw new RuntimeException("Client is not connected to any Elasticsearch nodes!"); 95 | } 96 | 97 | return client; 98 | } 99 | 100 | private void createIndexAndMapping(Client client) { 101 | // Create the Index and Mappings before indexing the entities: 102 | try { 103 | createIndex(client, getIndexName()); 104 | createMapping(client, getIndexName(), getMapping()); 105 | } catch (ResourceAlreadyExistsException e) { 106 | // The Index already exists! We shouldn't exit here, because we can 107 | // (more or less) safely assume, that the index has already been 108 | // created by some of the other worker processes. 109 | System.err.println(e.getMessage()); 110 | } 111 | } 112 | 113 | private void createIndex(Client client, String indexName) { 114 | if (!ElasticSearchUtils.indexExist(client, indexName).isExists()) { 115 | ElasticSearchUtils.createIndex(client, indexName); 116 | } 117 | } 118 | 119 | private void createMapping(Client client, String indexName, IElasticSearchMapping mapping) { 120 | if (ElasticSearchUtils.indexExist(client, indexName).isExists()) { 121 | ElasticSearchUtils.putMapping(client, indexName, mapping); 122 | } 123 | } 124 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sinks/elastic/LocalWeatherDataElasticSearchSink.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sinks.elastic; 5 | 6 | import de.bytefish.elasticutils.elasticsearch6.mapping.IElasticSearchMapping; 7 | 8 | public class LocalWeatherDataElasticSearchSink extends BaseElasticSearchSink { 9 | 10 | public LocalWeatherDataElasticSearchSink(String host, int port, int bulkSize) { 11 | super(host, port, bulkSize); 12 | } 13 | 14 | @Override 15 | protected String getIndexName() { 16 | return "weather_data"; 17 | } 18 | 19 | @Override 20 | protected IElasticSearchMapping getMapping() { 21 | return new elastic.mapping.LocalWeatherDataMapper(); 22 | } 23 | 24 | } 25 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sinks/pgsql/BasePostgresSink.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sinks.pgsql; 5 | 6 | import de.bytefish.pgbulkinsert.IPgBulkInsert; 7 | import de.bytefish.pgbulkinsert.pgsql.processor.BulkProcessor; 8 | import de.bytefish.pgbulkinsert.pgsql.processor.handler.BulkWriteHandler; 9 | import org.apache.flink.configuration.Configuration; 10 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 11 | import org.apache.flink.streaming.api.functions.sink.SinkFunction; 12 | import pgsql.connection.PooledConnectionFactory; 13 | 14 | import java.net.URI; 15 | 16 | public abstract class BasePostgresSink extends RichSinkFunction { 17 | 18 | private final URI databaseUri; 19 | private final int bulkSize; 20 | 21 | private BulkProcessor bulkProcessor; 22 | 23 | public BasePostgresSink(URI databaseUri, int bulkSize) { 24 | this.databaseUri = databaseUri; 25 | this.bulkSize = bulkSize; 26 | } 27 | 28 | @Override 29 | public void invoke(TEntity entity, SinkFunction.Context context) { 30 | bulkProcessor.add(entity); 31 | } 32 | 33 | @Override 34 | public void open(Configuration parameters) throws Exception { 35 | this.bulkProcessor = new BulkProcessor<>(new BulkWriteHandler<>(getBulkInsert(), new PooledConnectionFactory(databaseUri)), bulkSize); 36 | } 37 | 38 | @Override 39 | public void close() throws Exception { 40 | bulkProcessor.close(); 41 | } 42 | 43 | protected abstract IPgBulkInsert getBulkInsert(); 44 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sinks/pgsql/LocalWeatherDataPostgresSink.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sinks.pgsql; 5 | 6 | import de.bytefish.pgbulkinsert.IPgBulkInsert; 7 | import de.bytefish.pgbulkinsert.PgBulkInsert; 8 | import pgsql.mapping.LocalWeatherDataMapping; 9 | 10 | import java.net.URI; 11 | 12 | public class LocalWeatherDataPostgresSink extends BasePostgresSink { 13 | 14 | public LocalWeatherDataPostgresSink(URI databaseUri, int bulkSize) { 15 | super(databaseUri, bulkSize); 16 | } 17 | 18 | @Override 19 | protected IPgBulkInsert getBulkInsert() { 20 | return new PgBulkInsert(new LocalWeatherDataMapping("sample", "weather_data")); 21 | } 22 | 23 | } 24 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sources/csv/LocalWeatherDataSourceFunction.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sources.csv; 5 | 6 | import csv.parser.Parsers; 7 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 8 | import stream.sources.csv.converter.LocalWeatherDataConverter; 9 | 10 | import java.nio.charset.StandardCharsets; 11 | import java.nio.file.FileSystems; 12 | import java.nio.file.Path; 13 | import java.util.Iterator; 14 | import java.util.Map; 15 | import java.util.stream.Collectors; 16 | import java.util.stream.Stream; 17 | 18 | public class LocalWeatherDataSourceFunction implements SourceFunction { 19 | 20 | private volatile boolean isRunning = true; 21 | 22 | private String stationFilePath; 23 | private String localWeatherDataFilePath; 24 | 25 | public LocalWeatherDataSourceFunction(String stationFilePath, String localWeatherDataFilePath) { 26 | this.stationFilePath = stationFilePath; 27 | this.localWeatherDataFilePath = localWeatherDataFilePath; 28 | } 29 | 30 | @Override 31 | public void run(SourceFunction.SourceContext sourceContext) throws Exception { 32 | 33 | // The Source needs to be Serializable, so we have to construct the Paths at this point: 34 | final Path csvStationPath = FileSystems.getDefault().getPath(stationFilePath); 35 | final Path csvLocalWeatherDataPath = FileSystems.getDefault().getPath(localWeatherDataFilePath); 36 | 37 | // Get the Stream of LocalWeatherData Elements in the CSV File: 38 | try(Stream stream = getLocalWeatherData(csvStationPath, csvLocalWeatherDataPath)) { 39 | 40 | // We need to get an iterator, since the SourceFunction has to break out of its main loop on cancellation: 41 | Iterator iterator = stream.iterator(); 42 | 43 | // Make sure to cancel, when the Source function is canceled by an external event: 44 | while (isRunning && iterator.hasNext()) { 45 | sourceContext.collect(iterator.next()); 46 | } 47 | } 48 | } 49 | 50 | @Override 51 | public void cancel() { 52 | isRunning = false; 53 | } 54 | 55 | private Stream getLocalWeatherData(Path csvStationPath, Path csvLocalWeatherDataPath) { 56 | 57 | // A map between the WBAN and Station for faster Lookups: 58 | final Map stationMap = getStationMap(csvStationPath); 59 | 60 | // Turns the Stream of CSV data into the Elasticsearch representation: 61 | return getLocalWeatherData(csvLocalWeatherDataPath) 62 | // Only use Measurements with a Station: 63 | .filter(x -> stationMap.containsKey(x.getWban())) 64 | // And turn the Station and LocalWeatherData into the ElasticSearch representation: 65 | .map(x -> { 66 | // First get the matching Station: 67 | csv.model.Station station = stationMap.get(x.getWban()); 68 | // Convert to the Elastic Representation: 69 | return LocalWeatherDataConverter.convert(x, station); 70 | }); 71 | } 72 | 73 | private static Stream getLocalWeatherData(Path path) { 74 | return Parsers.LocalWeatherDataParser().readFromFile(path, StandardCharsets.US_ASCII) 75 | .filter(x -> x.isValid()) 76 | .map(x -> x.getResult()); 77 | } 78 | 79 | private static Stream getStations(Path path) { 80 | return Parsers.StationParser().readFromFile(path, StandardCharsets.US_ASCII) 81 | .filter(x -> x.isValid()) 82 | .map(x -> x.getResult()); 83 | } 84 | 85 | private Map getStationMap(Path path) { 86 | try (Stream stationStream = getStations(path)) { 87 | 88 | return stationStream 89 | // Group by WBAN, there are probably faulty entries with duplicate Station IDs: 90 | .collect(Collectors.groupingBy(x -> x.getWban())) 91 | // Turn into a Stream again: 92 | .entrySet().stream() 93 | // Make a best guess and take the first one: 94 | .collect(Collectors.toMap(x -> x.getKey(), x -> x.getValue().get(0))); 95 | } 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sources/csv/converter/LocalWeatherDataConverter.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sources.csv.converter; 5 | 6 | 7 | import java.time.LocalDate; 8 | import java.time.LocalTime; 9 | 10 | public class LocalWeatherDataConverter { 11 | 12 | public static model.LocalWeatherData convert(csv.model.LocalWeatherData csvLocalWeatherData, csv.model.Station csvStation) { 13 | 14 | LocalDate date = csvLocalWeatherData.getDate(); 15 | LocalTime time = csvLocalWeatherData.getTime(); 16 | String skyCondition = csvLocalWeatherData.getSkyCondition(); 17 | Float stationPressure = csvLocalWeatherData.getStationPressure(); 18 | Float temperature = csvLocalWeatherData.getDryBulbCelsius(); 19 | Float windSpeed = csvLocalWeatherData.getWindSpeed(); 20 | 21 | // Convert the Station data: 22 | model.Station station = convert(csvStation); 23 | 24 | return new model.LocalWeatherData(station, date, time, temperature, windSpeed, stationPressure, skyCondition); 25 | } 26 | 27 | public static model.Station convert(csv.model.Station csvStation) { 28 | String wban = csvStation.getWban(); 29 | String name = csvStation.getName(); 30 | String state = csvStation.getState(); 31 | String location = csvStation.getLocation(); 32 | Integer timeZone = csvStation.getTimeZone(); 33 | model.GeoLocation geoLocation = new model.GeoLocation(csvStation.getLatitude(), csvStation.getLongitude()); 34 | 35 | return new model.Station(wban, name, state, location, timeZone, geoLocation); 36 | } 37 | 38 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/stream/sources/periodic/PeriodicEmittingDataSourceFunction.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package stream.sources.periodic; 5 | 6 | import io.reactivex.Observable; 7 | import io.reactivex.annotations.NonNull; 8 | import io.reactivex.disposables.CompositeDisposable; 9 | import io.reactivex.disposables.Disposable; 10 | import io.reactivex.functions.BiFunction; 11 | import io.reactivex.functions.Consumer; 12 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 13 | 14 | import java.time.Duration; 15 | import java.util.concurrent.TimeUnit; 16 | 17 | /** 18 | * This SourceFunction allows to emit Events in a certain delay from a predefined Iterable. 19 | * 20 | * @param Event Type, which will be emitted 21 | */ 22 | public abstract class PeriodicEmittingDataSourceFunction implements SourceFunction { 23 | 24 | private CompositeDisposable disposables; 25 | 26 | @Override 27 | public void run(SourceContext sourceContext) throws Exception { 28 | 29 | disposables = new CompositeDisposable(); 30 | 31 | Observable 32 | // We zip two Streams: The Iterable and an Interval emitting data stream: 33 | .zip( 34 | Observable.fromIterable(iterable()), 35 | Observable.interval(interval().toMillis(), TimeUnit.MILLISECONDS), 36 | new BiFunction() { 37 | @Override 38 | public TEventType apply(@NonNull TEventType obs, @NonNull Long timer) throws Exception { 39 | return obs; 40 | } 41 | } 42 | ) 43 | // When the Subscription happens, add it to the list of Disposables: 44 | .doOnSubscribe(new Consumer() { 45 | @Override 46 | public void accept(Disposable disposable) throws Exception { 47 | disposables.add(disposable); 48 | } 49 | }) 50 | // We want this to be synchronous on the current thread, so do all this in a blocking subscribe: 51 | .blockingSubscribe(new Consumer() { 52 | @Override 53 | public void accept(TEventType event) throws Exception { 54 | sourceContext.collect(event); 55 | } 56 | }); 57 | } 58 | 59 | @Override 60 | public void cancel() { 61 | if(disposables != null) { 62 | disposables.clear(); 63 | } 64 | } 65 | 66 | protected abstract Iterable iterable(); 67 | 68 | protected abstract Duration interval(); 69 | } 70 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/utils/DateUtilities.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package utils; 5 | 6 | import java.time.LocalDate; 7 | import java.time.LocalDateTime; 8 | import java.time.LocalTime; 9 | import java.time.ZoneOffset; 10 | import java.time.OffsetDateTime; 11 | import java.util.Date; 12 | 13 | public class DateUtilities { 14 | 15 | public static Date from(LocalDate localDate, LocalTime localTime, ZoneOffset zoneOffset) { 16 | LocalDateTime localDateTime = localDate.atTime(localTime); 17 | 18 | return from(localDateTime, zoneOffset); 19 | } 20 | 21 | public static Date from(LocalDateTime localDateTime, ZoneOffset zoneOffset) { 22 | OffsetDateTime offsetDateTime = localDateTime.atOffset(zoneOffset); 23 | 24 | return Date.from(offsetDateTime.toInstant()); 25 | } 26 | 27 | } -------------------------------------------------------------------------------- /FlinkExperiments/src/main/java/utils/StringUtils.java: -------------------------------------------------------------------------------- 1 | // Copyright (c) Philipp Wagner. All rights reserved. 2 | // Licensed under the MIT license. See LICENSE file in the project root for full license information. 3 | 4 | package utils; 5 | 6 | public class StringUtils { 7 | 8 | private StringUtils() {} 9 | 10 | public static String StringEmpty = ""; 11 | 12 | public static boolean isNullOrWhiteSpace(String input) { 13 | return input == null || input.equals(StringEmpty) || input.trim().length() == 0; 14 | } 15 | 16 | } 17 | -------------------------------------------------------------------------------- /FlinkExperiments/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 18 | 19 | 20 | 21 | 22 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n 23 | 24 | 25 | 26 | 27 | 28 | 29 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) Philipp Wagner 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # FlinkExperiments # 2 | 3 | ## Project ## 4 | 5 | This project is a sample project for [Apache Flink]. The application parses the [Quality Controlled Local Climatological Data (QCLCD)] of 6 | March 2015, calculates the maximum daily temperature of the stream by using [Apache Flink] and writes the results back into an [Elasticsearch] 7 | and [PostgreSQL] database. 8 | 9 | ## Dataset ## 10 | 11 | The data is the [Quality Controlled Local Climatological Data (QCLCD)]: 12 | 13 | > Quality Controlled Local Climatological Data (QCLCD) consist of hourly, daily, and monthly summaries for approximately 14 | > 1,600 U.S. locations. Daily Summary forms are not available for all stations. Data are available beginning January 1, 2005 15 | > and continue to the present. Please note, there may be a 48-hour lag in the availability of the most recent data. 16 | 17 | The data is available at: 18 | 19 | * [http://www.ncdc.noaa.gov/orders/qclcd/](http://www.ncdc.noaa.gov/orders/qclcd/) 20 | 21 | ## Result ## 22 | 23 | 24 | Kibana Visualization of the Average Temperature 25 | 26 | 27 | ## Running the Examples ## 28 | 29 | The records in the [Quality Controlled Local Climatological Data (QCLCD)] dataset are not sorted by the timestamp. The dataset needs to be prepared first, so that all records are sorted ascending by the time of measurement. 30 | 31 | I have written a small application, that sorts the original CSV data by the measurement time: 32 | 33 | * [PrepareWeatherData.java](https://github.com/bytefish/FlinkExperiments/blob/master/FlinkExperiments/src/main/java/csv/sorting/PrepareWeatherData.java) 34 | 35 | The result is a sorted CSV file, which can be used to run the examples. 36 | 37 | ## Further Reading ## 38 | 39 | I have written several blog posts on [Apache Flink]: 40 | 41 | * [June 10, 2016 - Stream Data Processing with Apache Flink](http://bytefish.de/blog/stream_data_processing_flink/) 42 | * [July 03, 2016 - Building Applications with Apache Flink (Part 1): Dataset, Data Preparation and Building a Model](http://bytefish.de/blog/apache_flink_series_1) 43 | * [July 03, 2016 - Building Applications with Apache Flink (Part 2): Writing a custom SourceFunction for the CSV Data](http://bytefish.de/blog/apache_flink_series_2) 44 | * [July 03, 2016 - Building Applications with Apache Flink (Part 3): Stream Processing with the DataStream API](http://bytefish.de/blog/apache_flink_series_3) 45 | * [July 03, 2016 - Building Applications with Apache Flink (Part 4): Writing and Using a custom PostgreSQL SinkFunction](http://bytefish.de/blog/apache_flink_series_4) 46 | * [July 10, 2016 - Building Applications with Apache Flink (Part 5): Complex Event Processing with Apache Flink](http://bytefish.de/blog/apache_flink_series_5) 47 | 48 | [Apache Flink]: https://flink.apache.org 49 | [Elasticsearch]: https://www.elastic.co/products/elasticsearch 50 | [PostgreSQL]: https://www.postgresql.org 51 | [Quality Controlled Local Climatological Data (QCLCD)]: https://www.ncdc.noaa.gov/data-access/land-based-station-data/land-based-datasets/quality-controlled-local-climatological-data-qclcd 52 | --------------------------------------------------------------------------------