├── consumeTweets.sh ├── consumeTweetWordcount.sh ├── README.md ├── start-es-kibana.sh ├── etl.properties ├── .gitignore ├── startStreamingETL.sh ├── src └── main │ ├── resources │ └── log4j.properties │ └── java │ └── com │ └── dataartisans │ ├── ThroughputLogger.java │ ├── TwitterIntoKafka.java │ └── StreamingETL.java ├── LICENSE └── pom.xml /consumeTweets.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | /home/robert/flink-workdir/kafka_2.10-0.9.0.0/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic twitter -------------------------------------------------------------------------------- /consumeTweetWordcount.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | /home/robert/flink-workdir/kafka_2.10-0.9.0.0/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic twitter-wordcount -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Flink Streaming ETL Demo 2 | 3 | 4 | ## Local Paths 5 | 6 | Kibana: http://localhost:5601/app/kibana 7 | 8 | Elasticsearch HQ: http://localhost:9200/_plugin/hq/ 9 | -------------------------------------------------------------------------------- /start-es-kibana.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | /home/robert/flink-workdir/elasticsearch/elasticsearch-2.3.3/bin/elasticsearch & 4 | 5 | /home/robert/flink-workdir/elasticsearch/kibana-4.5.1-linux-x64/bin/kibana & 6 | 7 | 8 | -------------------------------------------------------------------------------- /etl.properties: -------------------------------------------------------------------------------- 1 | topic: twitter 2 | bootstrap.servers: localhost:9092 3 | auto.offset.reset: earliest 4 | 5 | wc-topic: twitter-wordcount 6 | 7 | 8 | # elastic search 9 | cluster.name: elasticsearch 10 | bulk.flush.max.actions: 1 -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | target/ 2 | pom.xml.tag 3 | pom.xml.releaseBackup 4 | pom.xml.versionsBackup 5 | pom.xml.next 6 | release.properties 7 | dependency-reduced-pom.xml 8 | buildNumber.properties 9 | .mvn/timing.properties 10 | .idea/ 11 | *.iml -------------------------------------------------------------------------------- /startStreamingETL.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # clean up rolling sink 4 | rm -r rolling-sink/* 5 | 6 | /home/robert/incubator-flink/build-target/bin/flink run -p 1 -c com.dataartisans.StreamingETL \ 7 | /home/robert/flink-workdir/flink-streaming-etl/target/flink-streaming-etl-1.0-SNAPSHOT.jar \ 8 | /home/robert/flink-workdir/flink-streaming-etl/etl.properties 9 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | log4j.rootLogger=INFO, console 20 | 21 | log4j.appender.console=org.apache.log4j.ConsoleAppender 22 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n 24 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/ThroughputLogger.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans; 2 | 3 | import org.apache.flink.api.common.functions.FlatMapFunction; 4 | import org.apache.flink.util.Collector; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | public class ThroughputLogger implements FlatMapFunction { 9 | 10 | private static final Logger LOG = LoggerFactory.getLogger(ThroughputLogger.class); 11 | 12 | private long totalReceived = 0; 13 | private long lastTotalReceived = 0; 14 | private long lastLogTimeMs = -1; 15 | private long logfreq; 16 | 17 | public ThroughputLogger(long logfreq) { 18 | this.logfreq = logfreq; 19 | } 20 | 21 | @Override 22 | public void flatMap(String element, Collector collector) throws Exception { 23 | totalReceived++; 24 | if (totalReceived % logfreq == 0) { 25 | // throughput over entire time 26 | long now = System.currentTimeMillis(); 27 | 28 | // throughput for the last "logfreq" elements 29 | if(lastLogTimeMs == -1) { 30 | // init (the first) 31 | lastLogTimeMs = now; 32 | lastTotalReceived = totalReceived; 33 | } else { 34 | long timeDiff = now - lastLogTimeMs; 35 | long elementDiff = totalReceived - lastTotalReceived; 36 | double ex = (1000/(double)timeDiff); 37 | LOG.info("During the last {} ms, we received {} elements. That's {} elements/second/core. {} MB/sec/core. GB received {}", 38 | timeDiff, elementDiff, elementDiff*ex, elementDiff*ex*element.length() / 1024 / 1024, (totalReceived * element.length()) / 1024 / 1024 / 1024); 39 | // reinit 40 | lastLogTimeMs = now; 41 | lastTotalReceived = totalReceived; 42 | } 43 | } 44 | } 45 | } -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/TwitterIntoKafka.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans; 2 | 3 | /** 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | import org.apache.flink.api.common.functions.FlatMapFunction; 22 | import org.apache.flink.api.java.utils.ParameterTool; 23 | import org.apache.flink.streaming.api.datastream.DataStream; 24 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 25 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; 26 | import org.apache.flink.streaming.connectors.twitter.TwitterSource; 27 | import org.apache.flink.streaming.util.serialization.SimpleStringSchema; 28 | import org.apache.flink.util.Collector; 29 | import org.codehaus.jackson.JsonNode; 30 | import org.codehaus.jackson.map.ObjectMapper; 31 | 32 | 33 | 34 | /** 35 | * Ingest data from Twitter into Kafka 36 | */ 37 | public class TwitterIntoKafka { 38 | 39 | public static void main(String[] args) throws Exception { 40 | // set up the streaming execution environment 41 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 42 | 43 | ParameterTool params = ParameterTool.fromPropertiesFile(args[0]); 44 | DataStream twitterStreamString = env.addSource(new TwitterSource(params.getProperties())); 45 | DataStream filteredStream = twitterStreamString.flatMap(new ParseJson()); 46 | filteredStream.flatMap(new ThroughputLogger(5000L)).setParallelism(1); 47 | 48 | filteredStream.addSink(new FlinkKafkaProducer09<>("twitter", new SimpleStringSchema(), params.getProperties())); 49 | 50 | // execute program 51 | env.execute("Ingest data from Twitter to Kafka"); 52 | } 53 | 54 | public static class ParseJson implements FlatMapFunction { 55 | private static final long serialVersionUID = 1L; 56 | 57 | private transient ObjectMapper jsonParser; 58 | /** 59 | * Select the language from the incoming JSON text 60 | */ 61 | @Override 62 | public void flatMap(String value, Collector out) throws Exception { 63 | if(jsonParser == null) { 64 | jsonParser = new ObjectMapper(); 65 | } 66 | JsonNode jsonNode = jsonParser.readValue(value, JsonNode.class); 67 | //boolean isEnglish = jsonNode.has("user") && jsonNode.get("user").has("lang") && jsonNode.get("user").get("lang").getValueAsText().equals("en"); 68 | boolean hasText = jsonNode.has("text"); 69 | if (hasText) { 70 | out.collect(value); 71 | } 72 | } 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 19 | 21 | 4.0.0 22 | 23 | com.dataartisans 24 | flink-streaming-etl 25 | 1.0-SNAPSHOT 26 | jar 27 | 28 | Flink Quickstart Job 29 | http://www.myorganization.org 30 | 31 | 32 | UTF-8 33 | 1.1-SNAPSHOT 34 | 35 | 36 | 37 | 38 | apache.snapshots 39 | Apache Development Snapshot Repository 40 | https://repository.apache.org/content/repositories/snapshots/ 41 | 42 | false 43 | 44 | 45 | true 46 | 47 | 48 | 49 | 50 | 51 | 52 | org.apache.flink 53 | flink-java 54 | ${flink.version} 55 | 56 | 57 | org.apache.flink 58 | flink-streaming-java_2.10 59 | ${flink.version} 60 | 61 | 62 | org.apache.flink 63 | flink-clients_2.10 64 | ${flink.version} 65 | 66 | 67 | 68 | org.apache.flink 69 | flink-connector-twitter_2.10 70 | ${flink.version} 71 | 72 | 73 | 74 | org.apache.flink 75 | flink-connector-kafka-0.9_2.10 76 | ${flink.version} 77 | 78 | 79 | 80 | org.apache.flink 81 | flink-connector-filesystem_2.10 82 | ${flink.version} 83 | 84 | 85 | 86 | org.apache.flink 87 | flink-connector-elasticsearch2_2.10 88 | ${flink.version} 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | build-jar 97 | 98 | false 99 | 100 | 101 | 102 | org.apache.flink 103 | flink-java 104 | ${flink.version} 105 | provided 106 | 107 | 108 | org.apache.flink 109 | flink-streaming-java_2.10 110 | ${flink.version} 111 | provided 112 | 113 | 114 | org.apache.flink 115 | flink-clients_2.10 116 | ${flink.version} 117 | provided 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | org.apache.maven.plugins 126 | maven-shade-plugin 127 | 2.4.1 128 | 129 | 130 | package 131 | 132 | shade 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | 148 | 149 | 152 | 153 | org.apache.maven.plugins 154 | maven-shade-plugin 155 | 2.4.1 156 | 157 | 158 | 159 | package 160 | 161 | shade 162 | 163 | 164 | 165 | 166 | 169 | org.apache.flink:flink-annotations 170 | org.apache.flink:flink-shaded-hadoop1 171 | org.apache.flink:flink-shaded-hadoop2 172 | org.apache.flink:flink-shaded-curator-recipes 173 | org.apache.flink:flink-core 174 | org.apache.flink:flink-java 175 | org.apache.flink:flink-scala_2.10 176 | org.apache.flink:flink-runtime_2.10 177 | org.apache.flink:flink-optimizer_2.10 178 | org.apache.flink:flink-clients_2.10 179 | org.apache.flink:flink-avro_2.10 180 | org.apache.flink:flink-examples-batch_2.10 181 | org.apache.flink:flink-examples-streaming_2.10 182 | org.apache.flink:flink-streaming-java_2.10 183 | 184 | 190 | org.scala-lang:scala-library 191 | org.scala-lang:scala-compiler 192 | org.scala-lang:scala-reflect 193 | com.amazonaws:aws-java-sdk 194 | com.typesafe.akka:akka-actor_* 195 | com.typesafe.akka:akka-remote_* 196 | com.typesafe.akka:akka-slf4j_* 197 | io.netty:netty-all 198 | io.netty:netty 199 | commons-fileupload:commons-fileupload 200 | org.apache.avro:avro 201 | commons-collections:commons-collections 202 | org.codehaus.jackson:jackson-core-asl 203 | org.codehaus.jackson:jackson-mapper-asl 204 | com.thoughtworks.paranamer:paranamer 205 | org.xerial.snappy:snappy-java 206 | org.apache.commons:commons-compress 207 | org.tukaani:xz 208 | com.esotericsoftware.kryo:kryo 209 | com.esotericsoftware.minlog:minlog 210 | org.objenesis:objenesis 211 | com.twitter:chill_* 212 | com.twitter:chill-java 213 | com.twitter:chill-avro_* 214 | com.twitter:chill-bijection_* 215 | com.twitter:bijection-core_* 216 | com.twitter:bijection-avro_* 217 | commons-lang:commons-lang 218 | junit:junit 219 | de.javakaffee:kryo-serializers 220 | org.apache.commons:commons-lang3 221 | org.slf4j:slf4j-api 222 | org.slf4j:slf4j-log4j12 223 | log4j:log4j 224 | org.apache.commons:commons-math 225 | org.apache.sling:org.apache.sling.commons.json 226 | commons-logging:commons-logging 227 | commons-codec:commons-codec 228 | com.fasterxml.jackson.core:jackson-core 229 | com.fasterxml.jackson.core:jackson-databind 230 | com.fasterxml.jackson.core:jackson-annotations 231 | stax:stax-api 232 | com.typesafe:config 233 | org.uncommons.maths:uncommons-maths 234 | com.github.scopt:scopt_* 235 | commons-io:commons-io 236 | commons-cli:commons-cli 237 | 238 | 239 | 240 | 241 | org.apache.flink:* 242 | 243 | 244 | org/apache/flink/shaded/com/** 245 | web-docs/** 246 | 247 | 248 | 249 | 251 | *:* 252 | 253 | META-INF/*.SF 254 | META-INF/*.DSA 255 | META-INF/*.RSA 256 | 257 | 258 | 259 | 260 | 261 | 262 | 264 | 271 | false 272 | 273 | 274 | 275 | 276 | 277 | 278 | org.apache.maven.plugins 279 | maven-compiler-plugin 280 | 3.1 281 | 282 | 1.8 283 | 1.8 284 | 285 | 286 | 287 | 288 | 289 | 290 | 350 | 351 | 352 | 353 | -------------------------------------------------------------------------------- /src/main/java/com/dataartisans/StreamingETL.java: -------------------------------------------------------------------------------- 1 | package com.dataartisans; 2 | 3 | import com.fasterxml.jackson.databind.node.ObjectNode; 4 | import com.google.common.collect.ImmutableList; 5 | import org.apache.flink.api.common.functions.FlatMapFunction; 6 | import org.apache.flink.api.common.functions.FoldFunction; 7 | import org.apache.flink.api.common.functions.RuntimeContext; 8 | import org.apache.flink.api.common.restartstrategy.RestartStrategies; 9 | import org.apache.flink.api.java.tuple.Tuple; 10 | import org.apache.flink.api.java.tuple.Tuple1; 11 | import org.apache.flink.api.java.tuple.Tuple2; 12 | import org.apache.flink.api.java.tuple.Tuple3; 13 | import org.apache.flink.api.java.utils.ParameterTool; 14 | import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; 15 | import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; 16 | import org.apache.flink.runtime.state.filesystem.FsStateBackend; 17 | import org.apache.flink.streaming.api.TimeCharacteristic; 18 | import org.apache.flink.streaming.api.datastream.DataStream; 19 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 20 | import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; 21 | import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; 22 | import org.apache.flink.streaming.api.functions.windowing.WindowFunction; 23 | import org.apache.flink.streaming.api.windowing.time.Time; 24 | import org.apache.flink.streaming.api.windowing.windows.TimeWindow; 25 | import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink; 26 | import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSinkFunction; 27 | import org.apache.flink.streaming.connectors.elasticsearch2.RequestIndexer; 28 | import org.apache.flink.streaming.connectors.fs.DateTimeBucketer; 29 | import org.apache.flink.streaming.connectors.fs.RollingSink; 30 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; 31 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; 32 | import org.apache.flink.streaming.util.serialization.JSONDeserializationSchema; 33 | import org.apache.flink.streaming.util.serialization.SerializationSchema; 34 | import org.apache.flink.util.Collector; 35 | import org.elasticsearch.action.ActionRequest; 36 | import org.elasticsearch.client.Requests; 37 | 38 | import java.net.InetAddress; 39 | import java.net.InetSocketAddress; 40 | import java.util.ArrayList; 41 | import java.util.Collections; 42 | import java.util.Date; 43 | import java.util.HashMap; 44 | import java.util.List; 45 | import java.util.Map; 46 | import java.util.Properties; 47 | import java.util.UUID; 48 | 49 | public class StreamingETL { 50 | public static void main(String[] args) throws Exception { 51 | // parse arguments 52 | ParameterTool params = ParameterTool.fromPropertiesFile(args[0]); 53 | 54 | // create streaming environment 55 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 56 | 57 | // enable event time processing 58 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 59 | 60 | // enable fault-tolerance 61 | env.enableCheckpointing(1000); 62 | 63 | // enable restarts 64 | env.setRestartStrategy(RestartStrategies.fixedDelayRestart(50, 500L)); 65 | 66 | env.setStateBackend(new FsStateBackend("file:///home/robert/flink-workdir/flink-streaming-etl/state-backend")); 67 | 68 | // run each operator separately 69 | env.disableOperatorChaining(); 70 | 71 | // get data from Kafka 72 | Properties kParams = params.getProperties(); 73 | kParams.setProperty("group.id", UUID.randomUUID().toString()); 74 | DataStream inputStream = env.addSource(new FlinkKafkaConsumer09<>(params.getRequired("topic"), new JSONDeserializationSchema(), kParams)).name("Kafka 0.9 Source") 75 | .assignTimestampsAndWatermarks(new BoundedOutOfOrdernessTimestampExtractor(Time.minutes(1L)) { 76 | @Override 77 | public long extractTimestamp(ObjectNode jsonNodes) { 78 | return jsonNodes.get("timestamp_ms").asLong(); 79 | } 80 | }).name("Timestamp extractor"); 81 | 82 | // filter out records without lang field 83 | DataStream tweetsWithLang = inputStream.filter(jsonNode -> jsonNode.has("user") && jsonNode.get("user").has("lang")).name("Filter records without 'lang' field"); 84 | 85 | // select only lang = "en" tweets 86 | DataStream englishTweets = tweetsWithLang.filter(jsonNode -> jsonNode.get("user").get("lang").asText().equals("en")).name("Select 'lang'=en tweets"); 87 | 88 | // write to file system 89 | RollingSink rollingSink = new RollingSink<>(params.get("sinkPath", "/home/robert/flink-workdir/flink-streaming-etl/rolling-sink")); 90 | rollingSink.setBucketer(new DateTimeBucketer("yyyy-MM-dd-HH-mm")); // do a bucket for each minute 91 | englishTweets.addSink(rollingSink).name("Rolling FileSystem Sink"); 92 | 93 | // build aggregates (count per language) using window (10 seconds tumbling): 94 | DataStream> languageCounts = tweetsWithLang.keyBy(jsonNode -> jsonNode.get("user").get("lang").asText()) 95 | .timeWindow(Time.seconds(10)) 96 | .apply(new Tuple3<>(0L, "", 0L), new JsonFoldCounter(), new CountEmitter()).name("Count per Langauage (10 seconds tumbling)"); 97 | 98 | // write window aggregate to ElasticSearch 99 | List transportNodes = ImmutableList.of(new InetSocketAddress(InetAddress.getByName("localhost"), 9300)); 100 | ElasticsearchSink> elasticsearchSink = new ElasticsearchSink<>(params.toMap(), transportNodes, new ESRequest()); 101 | 102 | languageCounts.addSink(elasticsearchSink).name("ElasticSearch2 Sink"); 103 | 104 | // word-count on the tweet stream 105 | DataStream>>> topWordCount = tweetsWithLang 106 | // get text from tweets 107 | .map(tweet -> tweet.get("text").asText()).name("Get text from Tweets") 108 | // split text into (word, 1) tuples 109 | .flatMap(new FlatMapFunction>() { 110 | @Override 111 | public void flatMap(String s, Collector> collector) throws Exception { 112 | String[] splits = s.split(" "); 113 | for (String sp : splits) { 114 | collector.collect(new Tuple2<>(sp, 1L)); 115 | } 116 | } 117 | }).name("Tokenize words") 118 | // group by word 119 | .keyBy(0) 120 | // build 1 min windows, compute every 10 seconds --> count word frequency 121 | .timeWindow(Time.minutes(1L), Time.seconds(10L)).apply(new WordCountingWindow()).name("Count word frequency (1 min, 10 sec sliding window)") 122 | // build top n every 10 seconds 123 | .timeWindowAll(Time.seconds(10L)).apply(new TopNWords(10)).name("TopN Window (10s)"); 124 | 125 | // write top Ns to Kafka topic 126 | topWordCount.addSink(new FlinkKafkaProducer09<>(params.getRequired("wc-topic"), new ListSerSchema(), params.getProperties())).name("Write topN to Kafka"); 127 | 128 | env.execute("Streaming ETL"); 129 | 130 | } 131 | 132 | private static class JsonFoldCounter implements FoldFunction> { 133 | @Override 134 | public Tuple3 fold(Tuple3 current, ObjectNode o) throws Exception { 135 | current.f0++; 136 | return current; 137 | } 138 | } 139 | 140 | private static class CountEmitter implements WindowFunction, Tuple3, String, TimeWindow> { 141 | @Override 142 | public void apply(String key, TimeWindow timeWindow, Iterable> iterable, Collector> collector) throws Exception { 143 | long count = iterable.iterator().next().f0; 144 | collector.collect(Tuple3.of(count, key, timeWindow.getStart())); 145 | } 146 | } 147 | 148 | private static class ESRequest implements ElasticsearchSinkFunction> { 149 | 150 | @Override 151 | public void process(Tuple3 result, RuntimeContext runtimeContext, RequestIndexer requestIndexer) { 152 | requestIndexer.add(createIndexRequest(result)); 153 | } 154 | 155 | private ActionRequest createIndexRequest(Tuple3 result) { 156 | Map json = new HashMap<>(); 157 | json.put("count", result.f0); 158 | json.put("lang", result.f1); 159 | json.put("window-start", result.f2); 160 | 161 | return Requests.indexRequest() 162 | .index("twitter-stats") 163 | .type("stats") 164 | .source(json); 165 | } 166 | } 167 | 168 | private static class WordCountingWindow implements WindowFunction, Tuple2, Tuple, TimeWindow> { 169 | @Override 170 | public void apply(Tuple key, TimeWindow timeWindow, Iterable> iterable, Collector> collector) throws Exception { 171 | long count = 0; 172 | for(Tuple2 e: iterable) { 173 | count += e.f1; 174 | } 175 | collector.collect(Tuple2.of(((Tuple1)key).f0, count)); 176 | } 177 | } 178 | 179 | private static class TopNWords implements AllWindowFunction, Tuple2>>, TimeWindow> { 180 | private final int n; 181 | 182 | public TopNWords(int n) { 183 | this.n = n; 184 | } 185 | 186 | @Override 187 | public void apply(TimeWindow timeWindow, Iterable> iterable, Collector>>> collector) throws Exception { 188 | // put words in list 189 | List> words = new ArrayList<>(); 190 | for(Tuple2 word: iterable) { 191 | words.add(word); 192 | } 193 | 194 | if(words.size() > 0) { 195 | // sort list 196 | Collections.sort(words, (o1, o2) -> -1*Long.compare(o1.f1, o2.f1)); 197 | // return top n 198 | List> sublist = new ArrayList<>(words.subList(0, Math.min(n, words.size()))); 199 | collector.collect(Tuple2.of(new Date(timeWindow.getStart()), sublist)); 200 | } 201 | } 202 | } 203 | 204 | private static class ListSerSchema implements SerializationSchema>>> { 205 | 206 | @Override 207 | public byte[] serialize(Tuple2>> tuple2) { 208 | return (tuple2.f0.toString() + " - " + tuple2.toString()).getBytes(); 209 | } 210 | } 211 | 212 | 213 | /** 214 | * Create ES Index: 215 | * 216 | * 217 | curl -XPUT "http://localhost:9200/twitter-stats/_mapping/stats" -d' 218 | { 219 | "stats" : { 220 | "properties" : { 221 | "count": {"type": "long"}, 222 | "lang": {"type": "string"}, 223 | "window-start": {"type": "date"} 224 | }, 225 | "_timestamp" : {"enabled" : true, "path" : "window-start", "store": "yes" } 226 | } 227 | }' 228 | 229 | 230 | Sample tweet: 231 | {"created_at":"Wed Jun 01 13:35:32 +0000 2016","id":738001034451156992,"id_str":"738001034451156992","text":"@ceyekku mau eek ak","source":"\u003ca href=\"http:\/\/twitter.com\/download\/android\" rel=\"nofollow\"\u003eTwitter for Android\u003c\/a\u003e","truncated":false,"in_reply_to_status_id":737989421799067648,"in_reply_to_status_id_str":"737989421799067648","in_reply_to_user_id":4201628843,"in_reply_to_user_id_str":"4201628843","in_reply_to_screen_name":"ceyekku","user":{"id":4292546423,"id_str":"4292546423","name":"binnie","screen_name":"machiattous","location":"benten ;","url":"http:\/\/twitter.com\/kayirrie","description":"bakpjm","protected":false,"verified":false,"followers_count":152,"friends_count":145,"listed_count":5,"favourites_count":2815,"statuses_count":19744,"created_at":"Fri Nov 27 04:32:27 +0000 2015","utc_offset":-25200,"time_zone":"Pacific Time (US & Canada)","geo_enabled":false,"lang":"en","contributors_enabled":false,"is_translator":false,"profile_background_color":"C0DEED","profile_background_image_url":"http:\/\/abs.twimg.com\/images\/themes\/theme1\/bg.png","profile_background_image_url_https":"https:\/\/abs.twimg.com\/images\/themes\/theme1\/bg.png","profile_background_tile":false,"profile_link_color":"0084B4","profile_sidebar_border_color":"C0DEED","profile_sidebar_fill_color":"DDEEF6","profile_text_color":"333333","profile_use_background_image":true,"profile_image_url":"http:\/\/pbs.twimg.com\/profile_images\/736159560688226304\/Xchm2Pnq_normal.jpg","profile_image_url_https":"https:\/\/pbs.twimg.com\/profile_images\/736159560688226304\/Xchm2Pnq_normal.jpg","profile_banner_url":"https:\/\/pbs.twimg.com\/profile_banners\/4292546423\/1464349616","default_profile":true,"default_profile_image":false,"following":null,"follow_request_sent":null,"notifications":null},"geo":null,"coordinates":null,"place":null,"contributors":null,"is_quote_status":false,"retweet_count":0,"favorite_count":0,"entities":{"hashtags":[],"urls":[],"user_mentions":[{"screen_name":"ceyekku","name":"CEYEK.","id":4201628843,"id_str":"4201628843","indices":[0,8]}],"symbols":[]},"favorited":false,"retweeted":false,"filter_level":"low","lang":"in","timestamp_ms":"1464788132666"} 232 | 233 | { 234 | "created_at":"Wed Jun 01 13:35:32 +0000 2016", 235 | "id":738001034451156992, 236 | "id_str":"738001034451156992", 237 | "text":"@ceyekku mau eek ak", 238 | "source":"\u003ca href=\"http:\/\/twitter.com\/download\/android\" rel=\"nofollow\"\u003eTwitter for Android\u003c\/a\u003e", 239 | "truncated":false, 240 | "in_reply_to_status_id":737989421799067648, 241 | "in_reply_to_status_id_str":"737989421799067648", 242 | "in_reply_to_user_id":4201628843, 243 | "in_reply_to_user_id_str":"4201628843", 244 | "in_reply_to_screen_name":"ceyekku", 245 | "user":{ 246 | "id":4292546423, 247 | "id_str":"4292546423", 248 | "name":"binnie", 249 | "screen_name":"machiattous", 250 | "location":"benten ;", 251 | "url":"http:\/\/twitter.com\/kayirrie", 252 | "description":"bakpjm", 253 | "protected":false, 254 | "verified":false, 255 | "followers_count":152, 256 | "friends_count":145, 257 | "listed_count":5, 258 | "favourites_count":2815, 259 | "statuses_count":19744, 260 | "created_at":"Fri Nov 27 04:32:27 +0000 2015", 261 | "utc_offset":-25200, 262 | "time_zone":"Pacific Time (US & Canada)", 263 | "geo_enabled":false, 264 | "lang":"en", 265 | "contributors_enabled":false, 266 | "is_translator":false, 267 | "profile_background_color":"C0DEED", 268 | "profile_background_image_url":"http:\/\/abs.twimg.com\/images\/themes\/theme1\/bg.png", 269 | "profile_background_image_url_https":"https:\/\/abs.twimg.com\/images\/themes\/theme1\/bg.png", 270 | "profile_background_tile":false, 271 | "profile_link_color":"0084B4", 272 | "profile_sidebar_border_color":"C0DEED", 273 | "profile_sidebar_fill_color":"DDEEF6", 274 | "profile_text_color":"333333", 275 | "profile_use_background_image":true, 276 | "profile_image_url":"http:\/\/pbs.twimg.com\/profile_images\/736159560688226304\/Xchm2Pnq_normal.jpg", 277 | "profile_image_url_https":"https:\/\/pbs.twimg.com\/profile_images\/736159560688226304\/Xchm2Pnq_normal.jpg", 278 | "profile_banner_url":"https:\/\/pbs.twimg.com\/profile_banners\/4292546423\/1464349616", 279 | "default_profile":true, 280 | "default_profile_image":false, 281 | "following":null, 282 | "follow_request_sent":null, 283 | "notifications":null 284 | }, 285 | "geo":null, 286 | "coordinates":null, 287 | "place":null, 288 | "contributors":null, 289 | "is_quote_status":false, 290 | "retweet_count":0, 291 | "favorite_count":0, 292 | "entities":{ 293 | "hashtags":[ 294 | 295 | ], 296 | "urls":[ 297 | 298 | ], 299 | "user_mentions":[ 300 | { 301 | "screen_name":"ceyekku", 302 | "name":"CEYEK.", 303 | "id":4201628843, 304 | "id_str":"4201628843", 305 | "indices":[ 306 | 0, 307 | 8 308 | ] 309 | } 310 | ], 311 | "symbols":[ 312 | 313 | ] 314 | }, 315 | "favorited":false, 316 | "retweeted":false, 317 | "filter_level":"low", 318 | "lang":"in", 319 | "timestamp_ms":"1464788132666" 320 | } 321 | 322 | */ 323 | } 324 | --------------------------------------------------------------------------------