├── LICENSE ├── README.md ├── books ├── Introduction_to_Apache_Flink_book.pdf ├── Learning_Apache_Flink.pdf ├── Stream_Processing_with_Apache_Flink.pdf └── Streaming_System.pdf ├── flink-learning-cep ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ └── cep │ │ └── Main.java │ └── resources │ ├── application.properties │ └── logback.xml ├── flink-learning-common ├── README.md ├── pom.xml └── src │ ├── main │ └── java │ │ └── com │ │ └── zhisheng │ │ └── common │ │ ├── constant │ │ └── PropertiesConstants.java │ │ ├── model │ │ ├── MetricEvent.java │ │ └── ProductEvent.java │ │ ├── schemas │ │ └── MetricSchema.java │ │ ├── utils │ │ ├── DateUtil.java │ │ ├── ExecutionEnvUtil.java │ │ ├── GsonUtil.java │ │ ├── HttpUtil.java │ │ ├── KafkaConfigUtil.java │ │ └── StringUtil.java │ │ └── watermarks │ │ └── MetricWatermark.java │ └── test │ └── java │ └── com │ └── zhisheng │ └── common │ └── utils │ └── DateUtilTests.java ├── flink-learning-connectors ├── README.md ├── flink-learning-connectors-activemq │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── activemq │ │ │ └── Main.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-akka │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── com │ │ └── zhisheng │ │ └── connectors │ │ └── akka │ │ ├── AkkaSource.java │ │ └── utils │ │ ├── ReceiverActor.java │ │ ├── SubscribeReceiver.java │ │ └── UnsubscribeReceiver.java ├── flink-learning-connectors-cassandra │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── cassandra │ │ │ └── Main.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-es6 │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── es6 │ │ │ ├── Main.java │ │ │ └── utils │ │ │ └── ElasticSearchSinkUtil.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-flume │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── flume │ │ │ ├── FlumeEventBuilder.java │ │ │ ├── FlumeSink.java │ │ │ ├── Main.java │ │ │ └── utils │ │ │ └── FlumeUtil.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-hbase │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── hbase │ │ │ ├── HBaseReadMain.java │ │ │ ├── HBaseStreamWriteMain.java │ │ │ ├── HBaseWriteMain.java │ │ │ ├── Main.java │ │ │ └── constant │ │ │ └── HBaseConstant.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-hdfs │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── hdfs │ │ │ └── Main.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-influxdb │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── influxdb │ │ │ ├── InfluxDBConfig.java │ │ │ ├── InfluxDBSink.java │ │ │ └── Main.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-kafka │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── kafka │ │ │ └── Main.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-kudu │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── com │ │ └── zhisheng │ │ └── connectors │ │ └── kudu │ │ ├── KuduInputFormat.java │ │ ├── KuduOutputFormat.java │ │ ├── KuduSink.java │ │ ├── connector │ │ ├── KuduColumnInfo.java │ │ ├── KuduConnector.java │ │ ├── KuduFilterInfo.java │ │ ├── KuduMapper.java │ │ ├── KuduRow.java │ │ ├── KuduRowIterator.java │ │ └── KuduTableInfo.java │ │ └── serde │ │ ├── DefaultSerDe.java │ │ ├── KuduDeserialization.java │ │ ├── KuduSerialization.java │ │ └── PojoSerDe.java ├── flink-learning-connectors-mysql │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── mysql │ │ │ ├── Main.java │ │ │ ├── model │ │ │ └── Student.java │ │ │ ├── sinks │ │ │ └── SinkToMySQL.java │ │ │ └── utils │ │ │ └── KafkaUtil.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-netty │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── com │ │ └── zhisheng │ │ └── connectors │ │ └── netty │ │ └── Main.java ├── flink-learning-connectors-rabbitmq │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── rabbitmq │ │ │ ├── Main.java │ │ │ ├── Main1.java │ │ │ ├── model │ │ │ └── EndPoint.java │ │ │ └── utils │ │ │ └── RabbitMQProducerUtil.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml ├── flink-learning-connectors-redis │ ├── README.md │ ├── pom.xml │ └── src │ │ ├── main │ │ ├── java │ │ │ └── com │ │ │ │ └── zhisheng │ │ │ │ └── connectors │ │ │ │ └── redis │ │ │ │ ├── Main.java │ │ │ │ └── utils │ │ │ │ └── ProductUtil.java │ │ └── resources │ │ │ ├── application.properties │ │ │ └── logback.xml │ │ └── test │ │ └── java │ │ └── RedisTest.java ├── flink-learning-connectors-rocketmq │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ ├── java │ │ └── com │ │ │ └── zhisheng │ │ │ └── connectors │ │ │ └── rocketmq │ │ │ ├── RocketMQConfig.java │ │ │ ├── RocketMQSink.java │ │ │ ├── RocketMQSource.java │ │ │ ├── RocketMQUtils.java │ │ │ ├── RunningChecker.java │ │ │ ├── common │ │ │ ├── selector │ │ │ │ ├── DefaultTopicSelector.java │ │ │ │ ├── SimpleTopicSelector.java │ │ │ │ └── TopicSelector.java │ │ │ └── serialization │ │ │ │ ├── KeyValueDeserializationSchema.java │ │ │ │ ├── KeyValueSerializationSchema.java │ │ │ │ ├── SimpleKeyValueDeserializationSchema.java │ │ │ │ └── SimpleKeyValueSerializationSchema.java │ │ │ └── example │ │ │ ├── RocketMQFlinkExample.java │ │ │ ├── SimpleConsumer.java │ │ │ └── SimpleProducer.java │ │ └── resources │ │ ├── application.properties │ │ └── logback.xml └── pom.xml ├── flink-learning-data-sinks ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ └── data │ │ └── sinks │ │ ├── Main.java │ │ ├── model │ │ └── Student.java │ │ ├── sinks │ │ └── SinkToMySQL.java │ │ └── utils │ │ └── KafkaUtil.java │ └── resources │ ├── application.properties │ ├── logback.xml │ └── student.sql ├── flink-learning-data-sources ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ └── data │ │ └── sources │ │ ├── Main.java │ │ ├── Main2.java │ │ ├── ScheduleMain.java │ │ ├── model │ │ ├── Rule.java │ │ └── Student.java │ │ ├── sources │ │ └── SourceFromMySQL.java │ │ └── utils │ │ ├── KafkaUtil.java │ │ └── MySQLUtil.java │ └── resources │ ├── application.properties │ ├── logback.xml │ ├── rule.sql │ └── student.sql ├── flink-learning-examples ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ └── examples │ │ ├── batch │ │ ├── accumulator │ │ │ ├── Main.java │ │ │ └── Main2.java │ │ └── wordcount │ │ │ └── Main.java │ │ ├── streaming │ │ ├── File │ │ │ └── Main.java │ │ ├── async │ │ │ └── AsyncIOExample.java │ │ ├── iteration │ │ │ ├── IterateExample.java │ │ │ └── util │ │ │ │ └── IterateExampleData.java │ │ ├── join │ │ │ ├── WindowJoin.java │ │ │ └── WindowJoinSampleData.java │ │ ├── sideoutput │ │ │ └── Main.java │ │ ├── socket │ │ │ └── Main.java │ │ └── wordcount │ │ │ └── Main.java │ │ └── util │ │ ├── MySQLUtil.java │ │ └── ThrottledIterator.java │ └── resources │ ├── log4j.properties │ └── logback.xml ├── flink-learning-monitor ├── README.md ├── flink-learning-monitor-alert │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── com │ │ └── zhisheng │ │ └── alert │ │ ├── model │ │ ├── AtMobiles.java │ │ ├── BaseMessage.java │ │ ├── Email.java │ │ ├── LinkMessage.java │ │ ├── MarkDownMessage.java │ │ ├── MessageType.java │ │ ├── TextMessage.java │ │ └── WorkNotify.java │ │ └── utils │ │ ├── DingDingAccessTokenUtil.java │ │ ├── DingDingGroupMsgUtil.java │ │ ├── DingDingWorkspaceNoticeUtil.java │ │ ├── EmailNoticeUtil.java │ │ ├── PhoneNoticeUtil.java │ │ └── SMSNoticeUtil.java ├── flink-learning-monitor-collector │ ├── README.md │ └── pom.xml ├── flink-learning-monitor-common │ ├── README.md │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── com │ │ └── zhisheng │ │ └── common │ │ ├── model │ │ ├── Job.java │ │ ├── JobStatus.java │ │ └── Task.java │ │ └── utils │ │ └── PropertiesUtil.java ├── flink-learning-monitor-dashboard │ ├── README.md │ └── pom.xml ├── flink-learning-monitor-storage │ ├── README.md │ └── pom.xml └── pom.xml ├── flink-learning-sql ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ ├── model │ │ └── WC.java │ │ ├── sql │ │ ├── Sort.java │ │ └── StreamSQLExample.java │ │ └── table │ │ ├── ExplainingTable.java │ │ └── WordCountTable.java │ └── resources │ ├── application.properties │ └── logback.xml ├── flink-learning-state ├── README.md └── pom.xml ├── flink-learning-template ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ └── template │ │ └── Main.java │ └── resources │ ├── application.properties │ └── logback.xml ├── flink-learning-window ├── README.md ├── pom.xml └── src │ └── main │ ├── java │ └── com │ │ └── zhisheng │ │ ├── constant │ │ └── WindowConstant.java │ │ └── window │ │ └── Main.java │ └── resources │ ├── application.properties │ └── logback.xml ├── paper └── paper.md ├── pics ├── Flink-code.png └── Flink-learning.png └── pom.xml /books/Introduction_to_Apache_Flink_book.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/books/Introduction_to_Apache_Flink_book.pdf -------------------------------------------------------------------------------- /books/Learning_Apache_Flink.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/books/Learning_Apache_Flink.pdf -------------------------------------------------------------------------------- /books/Stream_Processing_with_Apache_Flink.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/books/Stream_Processing_with_Apache_Flink.pdf -------------------------------------------------------------------------------- /books/Streaming_System.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/books/Streaming_System.pdf -------------------------------------------------------------------------------- /flink-learning-cep/README.md: -------------------------------------------------------------------------------- 1 | ## Flink CEP -------------------------------------------------------------------------------- /flink-learning-cep/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-cep 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | 22 | 23 | 24 | 25 | org.apache.maven.plugins 26 | maven-shade-plugin 27 | 3.1.0 28 | 29 | false 30 | 31 | 32 | 33 | package 34 | 35 | shade 36 | 37 | 38 | 39 | 40 | 41 | 43 | com.zhisheng.cep.Main 44 | 45 | 47 | reference.conf 48 | 49 | 50 | 51 | 52 | *:*:*:* 53 | 54 | META-INF/*.SF 55 | META-INF/*.DSA 56 | META-INF/*.RSA 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | -------------------------------------------------------------------------------- /flink-learning-cep/src/main/java/com/zhisheng/cep/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.cep; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.KafkaConfigUtil; 7 | import org.apache.flink.api.java.utils.ParameterTool; 8 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 9 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 10 | 11 | /** 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class Main { 16 | public static void main(String[] args) throws Exception{ 17 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 18 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 19 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 20 | data.print(); 21 | 22 | env.execute("flink learning cep"); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /flink-learning-cep/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metrics-group 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=alert-metrics 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-cep/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-common/README.md: -------------------------------------------------------------------------------- 1 | ### Flink-learning-common 2 | 3 | 这个模块存放通用的代码(实体类、工具类、常量类) -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/constant/PropertiesConstants.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.constant; 2 | 3 | 4 | /** 5 | * blog:http://www.54tianzhisheng.cn/ 6 | * 微信公众号:zhisheng 7 | */ 8 | public class PropertiesConstants { 9 | public static final String ZHISHENG = "zhisheng"; 10 | public static final String KAFKA_BROKERS = "kafka.brokers"; 11 | public static final String DEFAULT_KAFKA_BROKERS = "localhost:9092"; 12 | public static final String KAFKA_ZOOKEEPER_CONNECT = "kafka.zookeeper.connect"; 13 | public static final String DEFAULT_KAFKA_ZOOKEEPER_CONNECT = "localhost:2181"; 14 | public static final String KAFKA_GROUP_ID = "kafka.group.id"; 15 | public static final String DEFAULT_KAFKA_GROUP_ID = "zhisheng"; 16 | public static final String METRICS_TOPIC = "metrics.topic"; 17 | public static final String CONSUMER_FROM_TIME = "consumer.from.time"; 18 | public static final String STREAM_PARALLELISM = "stream.parallelism"; 19 | public static final String STREAM_SINK_PARALLELISM = "stream.sink.parallelism"; 20 | public static final String STREAM_DEFAULT_PARALLELISM = "stream.default.parallelism"; 21 | public static final String STREAM_CHECKPOINT_ENABLE = "stream.checkpoint.enable"; 22 | public static final String STREAM_CHECKPOINT_INTERVAL = "stream.checkpoint.interval"; 23 | public static final String PROPERTIES_FILE_NAME = "/application.properties"; 24 | 25 | //es config 26 | public static final String ELASTICSEARCH_BULK_FLUSH_MAX_ACTIONS = "elasticsearch.bulk.flush.max.actions"; 27 | public static final String ELASTICSEARCH_HOSTS = "elasticsearch.hosts"; 28 | 29 | //mysql 30 | public static final String MYSQL_DATABASE = "mysql.database"; 31 | public static final String MYSQL_HOST = "mysql.host"; 32 | public static final String MYSQL_PASSWORD = "mysql.password"; 33 | public static final String MYSQL_PORT = "mysql.port"; 34 | public static final String MYSQL_USERNAME = "mysql.username"; 35 | } 36 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/model/MetricEvent.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Builder; 5 | import lombok.Data; 6 | import lombok.NoArgsConstructor; 7 | 8 | import java.util.Map; 9 | 10 | /** 11 | * blog:http://www.54tianzhisheng.cn/ 12 | * 微信公众号:zhisheng 13 | */ 14 | 15 | @Data 16 | @Builder 17 | @AllArgsConstructor 18 | @NoArgsConstructor 19 | public class MetricEvent { 20 | 21 | /** 22 | * Metric name 23 | */ 24 | private String name; 25 | 26 | /** 27 | * Metric timestamp 28 | */ 29 | private Long timestamp; 30 | 31 | /** 32 | * Metric fields 33 | */ 34 | private Map fields; 35 | 36 | /** 37 | * Metric tags 38 | */ 39 | private Map tags; 40 | } 41 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/model/ProductEvent.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Builder; 5 | import lombok.Data; 6 | import lombok.NoArgsConstructor; 7 | 8 | import java.util.List; 9 | 10 | /** 11 | * Desc: 商品 12 | * Created by zhisheng on 2019-04-18 13 | * blog:http://www.54tianzhisheng.cn/ 14 | * 微信公众号:zhisheng 15 | */ 16 | @Data 17 | @Builder 18 | @AllArgsConstructor 19 | @NoArgsConstructor 20 | public class ProductEvent { 21 | 22 | /** 23 | * Product Id 24 | */ 25 | private Long id; 26 | 27 | /** 28 | * Product 类目 Id 29 | */ 30 | private Long categoryId; 31 | 32 | /** 33 | * Product 编码 34 | */ 35 | private String code; 36 | 37 | /** 38 | * Product 店铺 Id 39 | */ 40 | private Long shopId; 41 | 42 | /** 43 | * Product 店铺 name 44 | */ 45 | private String shopName; 46 | 47 | /** 48 | * Product 品牌 Id 49 | */ 50 | private Long brandId; 51 | 52 | /** 53 | * Product 品牌 name 54 | */ 55 | private String brandName; 56 | 57 | /** 58 | * Product name 59 | */ 60 | private String name; 61 | 62 | /** 63 | * Product 图片地址 64 | */ 65 | private String imageUrl; 66 | 67 | /** 68 | * Product 状态(1(上架),-1(下架),-2(冻结),-3(删除)) 69 | */ 70 | private int status; 71 | 72 | /** 73 | * Product 类型 74 | */ 75 | private int type; 76 | 77 | /** 78 | * Product 标签 79 | */ 80 | private List tags; 81 | 82 | /** 83 | * Product 价格(以分为单位) 84 | */ 85 | private Long price; 86 | } 87 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/schemas/MetricSchema.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.schemas; 2 | 3 | import com.google.gson.Gson; 4 | import com.zhisheng.common.model.MetricEvent; 5 | import org.apache.flink.api.common.serialization.DeserializationSchema; 6 | import org.apache.flink.api.common.serialization.SerializationSchema; 7 | import org.apache.flink.api.common.typeinfo.TypeInformation; 8 | 9 | import java.io.IOException; 10 | import java.nio.charset.Charset; 11 | 12 | /** 13 | * Metric Schema ,支持序列化和反序列化 14 | * 15 | * blog:http://www.54tianzhisheng.cn/ 16 | * 微信公众号:zhisheng 17 | * 18 | */ 19 | public class MetricSchema implements DeserializationSchema, SerializationSchema { 20 | 21 | private static final Gson gson = new Gson(); 22 | 23 | @Override 24 | public MetricEvent deserialize(byte[] bytes) throws IOException { 25 | return gson.fromJson(new String(bytes), MetricEvent.class); 26 | } 27 | 28 | @Override 29 | public boolean isEndOfStream(MetricEvent metricEvent) { 30 | return false; 31 | } 32 | 33 | @Override 34 | public byte[] serialize(MetricEvent metricEvent) { 35 | return gson.toJson(metricEvent).getBytes(Charset.forName("UTF-8")); 36 | } 37 | 38 | @Override 39 | public TypeInformation getProducedType() { 40 | return TypeInformation.of(MetricEvent.class); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/utils/ExecutionEnvUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.utils; 2 | 3 | import com.zhisheng.common.constant.PropertiesConstants; 4 | import org.apache.flink.api.common.restartstrategy.RestartStrategies; 5 | import org.apache.flink.api.java.utils.ParameterTool; 6 | import org.apache.flink.streaming.api.TimeCharacteristic; 7 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 8 | 9 | import java.io.IOException; 10 | import java.util.HashMap; 11 | import java.util.Map; 12 | 13 | /** 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | */ 17 | public class ExecutionEnvUtil { 18 | public static ParameterTool createParameterTool(final String[] args) throws Exception { 19 | return ParameterTool 20 | .fromPropertiesFile(ExecutionEnvUtil.class.getResourceAsStream(PropertiesConstants.PROPERTIES_FILE_NAME)) 21 | .mergeWith(ParameterTool.fromArgs(args)) 22 | .mergeWith(ParameterTool.fromSystemProperties()) 23 | .mergeWith(ParameterTool.fromMap(getenv())); 24 | } 25 | 26 | public static final ParameterTool PARAMETER_TOOL = createParameterTool(); 27 | 28 | private static ParameterTool createParameterTool() { 29 | try { 30 | return ParameterTool 31 | .fromPropertiesFile(ExecutionEnvUtil.class.getResourceAsStream(PropertiesConstants.PROPERTIES_FILE_NAME)) 32 | .mergeWith(ParameterTool.fromSystemProperties()) 33 | .mergeWith(ParameterTool.fromMap(getenv())); 34 | } catch (IOException e) { 35 | e.printStackTrace(); 36 | } 37 | return null; 38 | } 39 | 40 | public static StreamExecutionEnvironment prepare(ParameterTool parameterTool) throws Exception { 41 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 42 | env.setParallelism(parameterTool.getInt(PropertiesConstants.STREAM_PARALLELISM, 5)); 43 | env.getConfig().disableSysoutLogging(); 44 | env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)); 45 | if (parameterTool.getBoolean(PropertiesConstants.STREAM_CHECKPOINT_ENABLE, true)) { 46 | env.enableCheckpointing(parameterTool.getInt(PropertiesConstants.STREAM_CHECKPOINT_INTERVAL, 1000)); // create a checkpoint every 5 seconds 47 | } 48 | env.getConfig().setGlobalJobParameters(parameterTool); // make parameters available in the web interface 49 | env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); 50 | return env; 51 | } 52 | 53 | private static Map getenv() { 54 | Map map = new HashMap<>(); 55 | for (Map.Entry entry : System.getenv().entrySet()) { 56 | map.put(entry.getKey(), entry.getValue()); 57 | } 58 | return map; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/utils/GsonUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.utils; 2 | 3 | import com.google.gson.Gson; 4 | 5 | import java.nio.charset.Charset; 6 | 7 | /** 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public class GsonUtil { 12 | private final static Gson gson = new Gson(); 13 | 14 | public static T fromJson(String value, Class type) { 15 | return gson.fromJson(value, type); 16 | } 17 | 18 | public static String toJson(Object value) { 19 | return gson.toJson(value); 20 | } 21 | 22 | public static byte[] toJSONBytes(Object value) { 23 | return gson.toJson(value).getBytes(Charset.forName("UTF-8")); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/utils/StringUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.utils; 2 | 3 | /** 4 | * blog:http://www.54tianzhisheng.cn/ 5 | * 微信公众号:zhisheng 6 | */ 7 | public class StringUtil { 8 | /** 9 | * 判空 10 | * 11 | * @param str 12 | * @return 13 | */ 14 | public static boolean isEmpty(String str) { 15 | return str == null || str.trim().length() == 0; 16 | } 17 | 18 | /** 19 | * 判非空 20 | * 21 | * @param str 22 | * @return 23 | */ 24 | public static boolean isNotEmpty(String str) { 25 | return !isEmpty(str); 26 | } 27 | 28 | /** 29 | * 包含 30 | * 31 | * @param str1 32 | * @param str2 33 | * @return 34 | */ 35 | public static boolean isContains(String str1, String str2) { 36 | return str1.contains(str2); 37 | } 38 | 39 | /** 40 | * 不包含 41 | * 42 | * @param str1 43 | * @param str2 44 | * @return 45 | */ 46 | public static boolean isNotContains(String str1, String str2) { 47 | return !isContains(str1, str2); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-learning-common/src/main/java/com/zhisheng/common/watermarks/MetricWatermark.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.watermarks; 2 | 3 | import com.zhisheng.common.model.MetricEvent; 4 | import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; 5 | import org.apache.flink.streaming.api.watermark.Watermark; 6 | 7 | import javax.annotation.Nullable; 8 | 9 | /** 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class MetricWatermark implements AssignerWithPeriodicWatermarks { 14 | 15 | private long currentTimestamp = Long.MIN_VALUE; 16 | 17 | @Override 18 | public long extractTimestamp(MetricEvent metricEvent, long previousElementTimestamp) { 19 | if (metricEvent.getTimestamp() > currentTimestamp) { 20 | this.currentTimestamp = metricEvent.getTimestamp(); 21 | } 22 | return currentTimestamp; 23 | } 24 | 25 | @Nullable 26 | @Override 27 | public Watermark getCurrentWatermark() { 28 | long maxTimeLag = 5000; 29 | return new Watermark(currentTimestamp == Long.MIN_VALUE ? Long.MIN_VALUE : currentTimestamp - maxTimeLag); 30 | 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-learning-connectors/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connectors 2 | 3 | 暂时有这些 Cooncetor,其中这些并不是 Flink 自带的,需要自己定义,另外提供这些 Connector 的使用案例,大家可以参考。欢迎补充和点赞 4 | 5 | ```text 6 | . 7 | ├── flink-learning-connectors-activemq 8 | ├── flink-learning-connectors-akka 9 | ├── flink-learning-connectors-cassandra 10 | ├── flink-learning-connectors-es6 11 | ├── flink-learning-connectors-flume 12 | ├── flink-learning-connectors-hbase 13 | ├── flink-learning-connectors-hdfs 14 | ├── flink-learning-connectors-influxdb 15 | ├── flink-learning-connectors-kafka 16 | ├── flink-learning-connectors-kudu 17 | ├── flink-learning-connectors-mysql 18 | ├── flink-learning-connectors-netty 19 | ├── flink-learning-connectors-rabbitmq 20 | ├── flink-learning-connectors-redis 21 | └── flink-learning-connectors-rocketmq 22 | ``` -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-activemq/README.md: -------------------------------------------------------------------------------- 1 | 模版项目,不做任何代码编写,方便创建新的 module 时复制 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-activemq/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-activemq 13 | 14 | 15 | 16 | 17 | org.apache.maven.plugins 18 | maven-shade-plugin 19 | 3.1.0 20 | 21 | false 22 | 23 | 24 | 25 | package 26 | 27 | shade 28 | 29 | 30 | 31 | 32 | 33 | 35 | com.zhisheng.connectors.activemq.Main 36 | 37 | 39 | reference.conf 40 | 41 | 42 | 43 | 44 | *:*:*:* 45 | 46 | META-INF/*.SF 47 | META-INF/*.DSA 48 | META-INF/*.RSA 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-activemq/src/main/java/com/zhisheng/connectors/activemq/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.activemq; 2 | 3 | 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | 6 | public class Main { 7 | public static void main(String[] args) throws Exception { 8 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 9 | 10 | env.execute("flink learning project template"); 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-activemq/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-activemq/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-akka/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connector akka 2 | 3 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-akka/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-akka 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-akka/src/main/java/com/zhisheng/connectors/akka/utils/ReceiverActor.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.akka.utils; 2 | 3 | import akka.actor.ActorRef; 4 | import akka.actor.ActorSelection; 5 | import akka.actor.UntypedActor; 6 | import org.apache.flink.api.java.tuple.Tuple2; 7 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; 8 | 9 | import java.util.Iterator; 10 | 11 | /** 12 | * Desc: 13 | * Created by zhisheng on 2019-06-08 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | */ 17 | public class ReceiverActor extends UntypedActor { 18 | // --- Fields set by the constructor 19 | private final SourceContext ctx; 20 | 21 | private final String urlOfPublisher; 22 | 23 | private final boolean autoAck; 24 | 25 | // --- Runtime fields 26 | private ActorSelection remotePublisher; 27 | 28 | public ReceiverActor(SourceContext ctx, 29 | String urlOfPublisher, 30 | boolean autoAck) { 31 | this.ctx = ctx; 32 | this.urlOfPublisher = urlOfPublisher; 33 | this.autoAck = autoAck; 34 | } 35 | 36 | @Override 37 | public void preStart() throws Exception { 38 | remotePublisher = getContext().actorSelection(urlOfPublisher); 39 | remotePublisher.tell(new SubscribeReceiver(getSelf()), getSelf()); 40 | } 41 | 42 | @SuppressWarnings("unchecked") 43 | @Override 44 | public void onReceive(Object message) 45 | throws Exception { 46 | if (message instanceof Iterable) { 47 | collect((Iterable) message); 48 | } else if (message instanceof Tuple2) { 49 | Tuple2 messageTuple = (Tuple2) message; 50 | collect(messageTuple.f0, messageTuple.f1); 51 | } else { 52 | collect(message); 53 | } 54 | 55 | if (autoAck) { 56 | getSender().tell("ack", getSelf()); 57 | } 58 | } 59 | 60 | /** 61 | * To handle {@link Iterable} data 62 | * 63 | * @param data data received from feeder actor 64 | */ 65 | private void collect(Iterable data) { 66 | Iterator iterator = data.iterator(); 67 | while (iterator.hasNext()) { 68 | ctx.collect(iterator.next()); 69 | } 70 | } 71 | 72 | /** 73 | * To handle single data 74 | * @param data data received from feeder actor 75 | */ 76 | private void collect(Object data) { 77 | ctx.collect(data); 78 | } 79 | 80 | /** 81 | * To handle data with timestamp 82 | * 83 | * @param data data received from feeder actor 84 | * @param timestamp timestamp received from feeder actor 85 | */ 86 | private void collect(Object data, long timestamp) { 87 | ctx.collectWithTimestamp(data, timestamp); 88 | } 89 | 90 | @Override 91 | public void postStop() throws Exception { 92 | remotePublisher.tell(new UnsubscribeReceiver(ActorRef.noSender()), 93 | ActorRef.noSender()); 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-akka/src/main/java/com/zhisheng/connectors/akka/utils/SubscribeReceiver.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.akka.utils; 2 | 3 | import akka.actor.ActorRef; 4 | 5 | import java.io.Serializable; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-08 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class SubscribeReceiver implements Serializable { 14 | private static final long serialVersionUID = 1L; 15 | private ActorRef receiverActor; 16 | 17 | public SubscribeReceiver(ActorRef receiverActor) { 18 | this.receiverActor = receiverActor; 19 | } 20 | 21 | public void setReceiverActor(ActorRef receiverActor) { 22 | this.receiverActor = receiverActor; 23 | } 24 | 25 | public ActorRef getReceiverActor() { 26 | return receiverActor; 27 | } 28 | 29 | @Override 30 | public boolean equals(Object obj) { 31 | if (obj instanceof SubscribeReceiver) { 32 | SubscribeReceiver other = (SubscribeReceiver) obj; 33 | return other.canEquals(this) && super.equals(other) 34 | && receiverActor.equals(other.getReceiverActor()); 35 | } else { 36 | return false; 37 | } 38 | } 39 | 40 | public boolean canEquals(Object obj) { 41 | return obj instanceof SubscribeReceiver; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-akka/src/main/java/com/zhisheng/connectors/akka/utils/UnsubscribeReceiver.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.akka.utils; 2 | 3 | import akka.actor.ActorRef; 4 | 5 | import java.io.Serializable; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-08 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class UnsubscribeReceiver implements Serializable { 14 | private static final long serialVersionUID = 1L; 15 | private ActorRef receiverActor; 16 | 17 | public UnsubscribeReceiver(ActorRef receiverActor) { 18 | this.receiverActor = receiverActor; 19 | } 20 | 21 | public void setReceiverActor(ActorRef receiverActor) { 22 | this.receiverActor = receiverActor; 23 | } 24 | 25 | public ActorRef getReceiverActor() { 26 | return receiverActor; 27 | } 28 | 29 | 30 | @Override 31 | public boolean equals(Object obj) { 32 | if (obj instanceof UnsubscribeReceiver) { 33 | UnsubscribeReceiver other = (UnsubscribeReceiver) obj; 34 | return other.canEquals(this) && super.equals(other) 35 | && receiverActor.equals(other.getReceiverActor()); 36 | } else { 37 | return false; 38 | } 39 | } 40 | 41 | public boolean canEquals(Object obj) { 42 | return obj instanceof UnsubscribeReceiver; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-cassandra/README.md: -------------------------------------------------------------------------------- 1 | 模版项目,不做任何代码编写,方便创建新的 module 时复制 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-cassandra/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-cassandra 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-connector-cassandra_${scala.binary.version} 18 | ${flink.version} 19 | 20 | 21 | 22 | 23 | 24 | 25 | org.apache.maven.plugins 26 | maven-shade-plugin 27 | 3.1.0 28 | 29 | false 30 | 31 | 32 | 33 | package 34 | 35 | shade 36 | 37 | 38 | 39 | 40 | 41 | 43 | com.zhisheng.connectors.cassandra.Main 44 | 45 | 47 | reference.conf 48 | 49 | 50 | 51 | 52 | *:*:*:* 53 | 54 | META-INF/*.SF 55 | META-INF/*.DSA 56 | META-INF/*.RSA 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-cassandra/src/main/java/com/zhisheng/connectors/cassandra/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.cassandra; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.KafkaConfigUtil; 7 | import org.apache.flink.api.java.utils.ParameterTool; 8 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 9 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 10 | 11 | /** 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class Main { 16 | public static void main(String[] args) throws Exception{ 17 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 18 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 19 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 20 | data.print(); 21 | 22 | env.execute("flink learning connectors cassandra"); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-cassandra/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metrics-group 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=alert-metrics 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-cassandra/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-es6/README.md: -------------------------------------------------------------------------------- 1 | ## Flink connector ElasticSearch 6.x 2 | 3 | [http://www.54tianzhisheng.cn/2018/12/30/Flink-ElasticSearch-Sink/](http://www.54tianzhisheng.cn/2018/12/30/Flink-ElasticSearch-Sink/) -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-es6/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-es6 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-connector-elasticsearch6_${scala.binary.version} 18 | ${flink.version} 19 | 20 | 21 | 22 | 23 | 24 | 25 | org.apache.maven.plugins 26 | maven-shade-plugin 27 | 3.1.0 28 | 29 | false 30 | 31 | 32 | 33 | package 34 | 35 | shade 36 | 37 | 38 | 39 | 40 | 41 | 43 | com.zhisheng.connectors.es6.Main 44 | 45 | 47 | reference.conf 48 | 49 | 50 | 51 | 52 | *:*:*:* 53 | 54 | META-INF/*.SF 55 | META-INF/*.DSA 56 | META-INF/*.RSA 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-es6/src/main/java/com/zhisheng/connectors/es6/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.es6; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.GsonUtil; 7 | import com.zhisheng.common.utils.KafkaConfigUtil; 8 | import com.zhisheng.connectors.es6.utils.ElasticSearchSinkUtil; 9 | import lombok.extern.slf4j.Slf4j; 10 | import org.apache.flink.api.common.functions.RuntimeContext; 11 | import org.apache.flink.api.java.utils.ParameterTool; 12 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 13 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 14 | import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; 15 | import org.apache.http.HttpHost; 16 | import org.elasticsearch.client.Requests; 17 | import org.elasticsearch.common.xcontent.XContentType; 18 | 19 | import java.util.List; 20 | 21 | import static com.zhisheng.common.constant.PropertiesConstants.*; 22 | 23 | /** 24 | * blog:http://www.54tianzhisheng.cn/ 25 | * 微信公众号:zhisheng 26 | */ 27 | @Slf4j 28 | public class Main { 29 | public static void main(String[] args) throws Exception { 30 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 31 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 32 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 33 | 34 | List esAddresses = ElasticSearchSinkUtil.getEsAddresses(parameterTool.get(ELASTICSEARCH_HOSTS)); 35 | int bulkSize = parameterTool.getInt(ELASTICSEARCH_BULK_FLUSH_MAX_ACTIONS, 40); 36 | int sinkParallelism = parameterTool.getInt(STREAM_SINK_PARALLELISM, 5); 37 | 38 | log.info("-----esAddresses = {}, parameterTool = {}, ", esAddresses, parameterTool); 39 | 40 | ElasticSearchSinkUtil.addSink(esAddresses, bulkSize, sinkParallelism, data, 41 | (MetricEvent metric, RuntimeContext runtimeContext, RequestIndexer requestIndexer) -> { 42 | requestIndexer.add(Requests.indexRequest() 43 | .index(ZHISHENG + "_" + metric.getName()) 44 | .type(ZHISHENG) 45 | .source(GsonUtil.toJSONBytes(metric), XContentType.JSON)); 46 | }); 47 | env.execute("flink learning connectors es6"); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-es6/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metrics-group-test 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=alert-metrics 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false 8 | elasticsearch.hosts=localhost:9201,localhost:9202 9 | elasticsearch.bulk.flush.max.actions=40 10 | stream.sink.parallelism=5 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-es6/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connector Flume -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-flume 13 | 14 | 15 | 1.9.0 16 | 17 | 18 | 19 | 20 | 21 | org.apache.flume 22 | flume-ng-core 23 | ${flume-ng.version} 24 | 25 | 26 | 27 | 28 | 29 | 30 | org.apache.maven.plugins 31 | maven-shade-plugin 32 | 3.1.0 33 | 34 | false 35 | 36 | 37 | 38 | package 39 | 40 | shade 41 | 42 | 43 | 44 | 45 | 46 | 48 | com.zhisheng.connectors.flume.Main 49 | 50 | 52 | reference.conf 53 | 54 | 55 | 56 | 57 | *:*:*:* 58 | 59 | META-INF/*.SF 60 | META-INF/*.DSA 61 | META-INF/*.RSA 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/src/main/java/com/zhisheng/connectors/flume/FlumeEventBuilder.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.flume; 2 | 3 | import org.apache.flink.api.common.functions.Function; 4 | import org.apache.flink.api.common.functions.RuntimeContext; 5 | import org.apache.flume.Event; 6 | 7 | import java.io.Serializable; 8 | 9 | /** 10 | * Desc: A function that can create a Event from an incoming instance of the given type. 11 | * Created by zhisheng on 2019-05-04 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public interface FlumeEventBuilder extends Function, Serializable { 16 | 17 | Event createFlumeEvent(IN value, RuntimeContext ctx); 18 | 19 | } 20 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/src/main/java/com/zhisheng/connectors/flume/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.flume; 2 | 3 | 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | 6 | public class Main { 7 | public static void main(String[] args) throws Exception { 8 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 9 | 10 | env.execute("flink learning project template"); 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/src/main/java/com/zhisheng/connectors/flume/utils/FlumeUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.flume.utils; 2 | 3 | import org.apache.flume.api.RpcClient; 4 | import org.apache.flume.api.RpcClientConfigurationConstants; 5 | import org.apache.flume.api.RpcClientFactory; 6 | 7 | import java.util.Properties; 8 | 9 | /** 10 | * Desc: 11 | * Created by zhisheng on 2019-05-04 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class FlumeUtil { 16 | private static final String CLIENT_TYPE_KEY = "client.type"; 17 | private static final String CLIENT_TYPE_DEFAULT_FAILOVER = "default_failover"; 18 | private static final String CLIENT_TYPE_DEFAULT_LOADBALANCING = "default_loadbalance"; 19 | 20 | public static RpcClient getRpcClient(String clientType, String hostname, Integer port, Integer batchSize) { 21 | Properties props; 22 | RpcClient client; 23 | switch(clientType.toUpperCase()) { 24 | case "THRIFT": 25 | client = RpcClientFactory.getThriftInstance(hostname, port, batchSize); 26 | break; 27 | case "DEFAULT": 28 | client = RpcClientFactory.getDefaultInstance(hostname, port, batchSize); 29 | break; 30 | case "DEFAULT_FAILOVER": 31 | props = getDefaultProperties(hostname, port, batchSize); 32 | props.put(CLIENT_TYPE_KEY, CLIENT_TYPE_DEFAULT_FAILOVER); 33 | client = RpcClientFactory.getInstance(props); 34 | break; 35 | case "DEFAULT_LOADBALANCE": 36 | props = getDefaultProperties(hostname, port, batchSize); 37 | props.put(CLIENT_TYPE_KEY, CLIENT_TYPE_DEFAULT_LOADBALANCING); 38 | client = RpcClientFactory.getInstance(props); 39 | break; 40 | default: 41 | throw new IllegalStateException("Unsupported client type - cannot happen"); 42 | } 43 | return client; 44 | } 45 | 46 | public static void destroy(RpcClient client) { 47 | if (null != client) { 48 | client.close(); 49 | } 50 | } 51 | 52 | private static Properties getDefaultProperties(String hostname, Integer port, Integer batchSize) { 53 | Properties props = new Properties(); 54 | props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS, "h1"); 55 | props.setProperty(RpcClientConfigurationConstants.CONFIG_HOSTS_PREFIX + "h1", 56 | hostname + ":" + port.intValue()); 57 | props.setProperty(RpcClientConfigurationConstants.CONFIG_BATCH_SIZE, batchSize.toString()); 58 | return props; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-flume/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hbase/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connector HBase -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hbase/src/main/java/com/zhisheng/connectors/hbase/HBaseReadMain.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.hbase; 2 | 3 | import org.apache.flink.addons.hbase.TableInputFormat; 4 | import org.apache.flink.api.common.functions.FilterFunction; 5 | import org.apache.flink.api.java.ExecutionEnvironment; 6 | import org.apache.flink.api.java.tuple.Tuple2; 7 | import org.apache.flink.configuration.ConfigConstants; 8 | import org.apache.hadoop.hbase.client.Result; 9 | import org.apache.hadoop.hbase.client.Scan; 10 | import org.apache.hadoop.hbase.util.Bytes; 11 | 12 | /** 13 | * Desc: 读取 HBase 数据 14 | * Created by zhisheng on 2019-05-04 15 | * blog:http://www.54tianzhisheng.cn/ 16 | * 微信公众号:zhisheng 17 | */ 18 | public class HBaseReadMain { 19 | //表名 20 | public static final String HBASE_TABLE_NAME = "zhisheng"; 21 | // 列族 22 | static final byte[] INFO = "info".getBytes(ConfigConstants.DEFAULT_CHARSET); 23 | //列名 24 | static final byte[] BAR = "bar".getBytes(ConfigConstants.DEFAULT_CHARSET); 25 | 26 | 27 | public static void main(String[] args) throws Exception { 28 | ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); 29 | env.createInput(new TableInputFormat>() { 30 | 31 | private Tuple2 reuse = new Tuple2(); 32 | 33 | @Override 34 | protected Scan getScanner() { 35 | Scan scan = new Scan(); 36 | scan.addColumn(INFO, BAR); 37 | return scan; 38 | } 39 | 40 | @Override 41 | protected String getTableName() { 42 | return HBASE_TABLE_NAME; 43 | } 44 | 45 | @Override 46 | protected Tuple2 mapResultToTuple(Result result) { 47 | String key = Bytes.toString(result.getRow()); 48 | String val = Bytes.toString(result.getValue(INFO, BAR)); 49 | reuse.setField(key, 0); 50 | reuse.setField(val, 1); 51 | return reuse; 52 | } 53 | }).filter(new FilterFunction>() { 54 | @Override 55 | public boolean filter(Tuple2 value) throws Exception { 56 | return value.f1.startsWith("zhisheng"); 57 | } 58 | }).print(); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hbase/src/main/java/com/zhisheng/connectors/hbase/constant/HBaseConstant.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.hbase.constant; 2 | 3 | 4 | public class HBaseConstant { 5 | 6 | public static final String HBASE_ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum"; 7 | public static final String HBASE_CLIENT_RETRIES_NUMBER = "hbase.client.retries.number"; 8 | public static final String HBASE_MASTER_INFO_PORT = "hbase.master.info.port"; 9 | public static final String HBASE_ZOOKEEPER_PROPERTY_CLIENTPORT = "hbase.zookeeper.property.clientPort"; 10 | public static final String HBASE_RPC_TIMEOUT = "hbase.rpc.timeout"; 11 | public static final String HBASE_CLIENT_OPERATION_TIMEOUT = "hbase.client.operation.timeout"; 12 | public static final String HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD = "hbase.client.scanner.timeout.period"; 13 | 14 | public static final String HBASE_TABLE_NAME = "hbase.table.name"; 15 | public static final String HBASE_COLUMN_NAME = "hbase.column.name"; 16 | } 17 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hbase/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false 10 | 11 | # HBase 12 | hbase.zookeeper.quorum=localhost:2181 13 | hbase.client.retries.number=1 14 | hbase.master.info.port=-1 15 | hbase.zookeeper.property.clientPort=2081 16 | hbase.rpc.timeout=30000 17 | hbase.client.operation.timeout=30000 18 | hbase.client.scanner.timeout.period=30000 19 | 20 | # HBase table name 21 | hbase.table.name=zhisheng_stream 22 | hbase.column.name=info_stream -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hbase/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hdfs/README.md: -------------------------------------------------------------------------------- 1 | 模版项目,不做任何代码编写,方便创建新的 module 时复制 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hdfs/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-hdfs 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-connector-filesystem_${scala.binary.version} 18 | ${flink.version} 19 | 20 | 21 | 22 | 23 | 24 | 25 | org.apache.maven.plugins 26 | maven-shade-plugin 27 | 3.1.0 28 | 29 | false 30 | 31 | 32 | 33 | package 34 | 35 | shade 36 | 37 | 38 | 39 | 40 | 41 | 43 | com.zhisheng.connectors.hdfs.Main 44 | 45 | 47 | reference.conf 48 | 49 | 50 | 51 | 52 | *:*:*:* 53 | 54 | META-INF/*.SF 55 | META-INF/*.DSA 56 | META-INF/*.RSA 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hdfs/src/main/java/com/zhisheng/connectors/hdfs/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.hdfs; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.KafkaConfigUtil; 7 | import lombok.extern.slf4j.Slf4j; 8 | import org.apache.flink.api.java.utils.ParameterTool; 9 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 10 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 11 | 12 | /** 13 | * blog:http://www.54tianzhisheng.cn/ 14 | * 微信公众号:zhisheng 15 | */ 16 | @Slf4j 17 | public class Main { 18 | public static void main(String[] args) throws Exception { 19 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 20 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 21 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 22 | 23 | 24 | env.execute("flink learning connectors hdfs"); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hdfs/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-hdfs/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connector influxDB -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-influxdb 13 | 14 | 2.7 15 | 16 | 17 | 18 | 19 | org.influxdb 20 | influxdb-java 21 | ${influxdb-client.version} 22 | 23 | 24 | 25 | 26 | 27 | 28 | org.apache.maven.plugins 29 | maven-shade-plugin 30 | 3.1.0 31 | 32 | false 33 | 34 | 35 | 36 | package 37 | 38 | shade 39 | 40 | 41 | 42 | 43 | 44 | 46 | com.zhisheng.connectors.influxdb.Main 47 | 48 | 50 | reference.conf 51 | 52 | 53 | 54 | 55 | *:*:*:* 56 | 57 | META-INF/*.SF 58 | META-INF/*.DSA 59 | META-INF/*.RSA 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/src/main/java/com/zhisheng/connectors/influxdb/InfluxDBConfig.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.influxdb; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Builder; 5 | import lombok.Data; 6 | import lombok.NoArgsConstructor; 7 | 8 | import java.io.Serializable; 9 | import java.util.concurrent.TimeUnit; 10 | 11 | /** 12 | * Desc: InfluxDB 配置 13 | * Created by zhisheng on 2019-05-01 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | */ 17 | @Data 18 | @Builder 19 | @NoArgsConstructor 20 | @AllArgsConstructor 21 | public class InfluxDBConfig implements Serializable { 22 | 23 | private static final long serialVersionUID = 1L; 24 | 25 | /** 26 | * 默认每次批处理的数据条数为 2000 条 27 | */ 28 | private static final int DEFAULT_BATCH_ACTIONS = 2000; 29 | 30 | /** 31 | * 默认每隔 100 32 | */ 33 | private static final int DEFAULT_FLUSH_DURATION = 100; 34 | 35 | /** 36 | * 数据库地址 37 | */ 38 | private String url; 39 | 40 | /** 41 | * 数据库用户名 42 | */ 43 | private String username; 44 | 45 | /** 46 | * 数据库密码 47 | */ 48 | private String password; 49 | 50 | /** 51 | * 数据库名 52 | */ 53 | private String database; 54 | 55 | /** 56 | * batch 57 | */ 58 | private int batchActions = DEFAULT_BATCH_ACTIONS; 59 | 60 | /** 61 | * flush duration 62 | */ 63 | private int flushDuration = DEFAULT_FLUSH_DURATION; 64 | 65 | /** 66 | * 单位 67 | */ 68 | private TimeUnit flushDurationTimeUnit = TimeUnit.MILLISECONDS; 69 | 70 | /** 71 | * 是否开启 GZIP 压缩 72 | */ 73 | private boolean enableGzip = false; 74 | 75 | /** 76 | * 是否创建数据库 77 | */ 78 | private boolean createDatabase = false; 79 | } 80 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/src/main/java/com/zhisheng/connectors/influxdb/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.influxdb; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.KafkaConfigUtil; 7 | import org.apache.flink.api.java.utils.ParameterTool; 8 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 9 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 10 | 11 | public class Main { 12 | public static void main(String[] args) throws Exception { 13 | ParameterTool parameterTool = ExecutionEnvUtil.PARAMETER_TOOL; 14 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 15 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 16 | 17 | //请将下面的这些字段弄成常量 18 | InfluxDBConfig config = InfluxDBConfig.builder() 19 | .url(parameterTool.get("influxdb.url")) 20 | .username(parameterTool.get("influxdb.username")) 21 | .password(parameterTool.get("influxdb.password")) 22 | .database(parameterTool.get("influxdb.database")) 23 | .batchActions(parameterTool.getInt("influxdb.batchActions")) 24 | .flushDuration(parameterTool.getInt("influxdb.flushDuration")) 25 | .enableGzip(parameterTool.getBoolean("influxdb.enableGzip")) 26 | .createDatabase(parameterTool.getBoolean("influxdb.createDatabase")) 27 | .build(); 28 | 29 | data.addSink(new InfluxDBSink(config)); 30 | 31 | env.execute("flink InfluxDB connector"); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false 10 | 11 | # influxDB 12 | 13 | influxdb.url=http://localhost:8086 14 | influxdb.username=root 15 | influxdb.password=root 16 | influxdb.database=metric-db 17 | influxdb.batchActions=2000 18 | influxdb.flushDuration=100 19 | influxdb.flushDurationTimeUnit=MILLISECONDS 20 | influxdb.enableGzip=false 21 | influxdb.createDatabase=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-influxdb/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kafka/README.md: -------------------------------------------------------------------------------- 1 | ## Flink connector Kafka 2 | 3 | [http://www.54tianzhisheng.cn/2019/01/06/Flink-Kafka-sink/](http://www.54tianzhisheng.cn/2019/01/06/Flink-Kafka-sink/) -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kafka/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-kafka 13 | 14 | 15 | 16 | 17 | org.apache.maven.plugins 18 | maven-shade-plugin 19 | 3.1.0 20 | 21 | false 22 | 23 | 24 | 25 | package 26 | 27 | shade 28 | 29 | 30 | 31 | 32 | 33 | 35 | com.zhisheng.connectors.kafka.Main 36 | 37 | 39 | reference.conf 40 | 41 | 42 | 43 | 44 | *:*:*:* 45 | 46 | META-INF/*.SF 47 | META-INF/*.DSA 48 | META-INF/*.RSA 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kafka/src/main/java/com/zhisheng/connectors/kafka/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kafka; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.schemas.MetricSchema; 6 | import com.zhisheng.common.utils.ExecutionEnvUtil; 7 | import com.zhisheng.common.utils.KafkaConfigUtil; 8 | import org.apache.flink.api.java.utils.ParameterTool; 9 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 10 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 11 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; 12 | 13 | /** 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | */ 17 | public class Main { 18 | public static void main(String[] args) throws Exception { 19 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 20 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 21 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 22 | 23 | data.addSink(new FlinkKafkaProducer011<>( 24 | parameterTool.get("kafka.sink.brokers"), 25 | parameterTool.get("kafka.sink.topic"), 26 | new MetricSchema() 27 | )).name("flink-connectors-kafka") 28 | .setParallelism(parameterTool.getInt("stream.sink.parallelism")); 29 | 30 | env.execute("flink learning connectors kafka"); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kafka/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=xxx:9092,xxx:9092,xxx:9092 2 | kafka.group.id=metrics-group-test 3 | kafka.zookeeper.connect=xxx:2181 4 | metrics.topic=xxx 5 | kafka.sink.brokers=localhost:9092 6 | kafka.sink.topic=metric-test 7 | stream.parallelism=5 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false 10 | stream.sink.parallelism=5 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kafka/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connectors Kudu 2 | 3 | https://github.com/apache/bahir-flink/blob/master/flink-connector-kudu/README.md 4 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-kudu 13 | 14 | 15 | 1.9.0 16 | 17 | 18 | 19 | 20 | 21 | com.zhisheng.flink 22 | flink-learning-common 23 | 1.0-SNAPSHOT 24 | 25 | 26 | org.apache.kudu 27 | kudu-client 28 | ${kudu.version} 29 | 30 | 31 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/src/main/java/com/zhisheng/connectors/kudu/connector/KuduRow.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kudu.connector; 2 | 3 | import org.apache.flink.types.Row; 4 | 5 | import java.lang.reflect.Field; 6 | import java.lang.reflect.Modifier; 7 | import java.util.Arrays; 8 | import java.util.Comparator; 9 | import java.util.LinkedHashMap; 10 | import java.util.Map; 11 | import java.util.stream.Stream; 12 | /** 13 | * Desc: 14 | * Created by zhisheng on 2019-06-08 15 | * blog:http://www.54tianzhisheng.cn/ 16 | * 微信公众号:zhisheng 17 | */ 18 | public class KuduRow extends Row { 19 | 20 | private Map rowNames; 21 | 22 | public KuduRow(Integer arity) { 23 | super(arity); 24 | rowNames = new LinkedHashMap<>(); 25 | } 26 | 27 | public Object getField(String name) { 28 | return super.getField(rowNames.get(name)); 29 | } 30 | 31 | public void setField(int pos, String name, Object value) { 32 | super.setField(pos, value); 33 | this.rowNames.put(name, pos); 34 | } 35 | 36 | public boolean isNull(String name) { 37 | return isNull(rowNames.get(name)); 38 | } 39 | 40 | public boolean isNull(int pos) { 41 | return getField(pos) == null; 42 | } 43 | 44 | private static int validFields(Object object) { 45 | Long validField = 0L; 46 | for (Class c = object.getClass(); c != null; c = c.getSuperclass()) { 47 | validField += basicValidation(c.getDeclaredFields()).count(); 48 | } 49 | return validField.intValue(); 50 | } 51 | 52 | private static Stream basicValidation(Field[] fields) { 53 | return Arrays.stream(fields) 54 | .filter(cField -> !Modifier.isStatic(cField.getModifiers())) 55 | .filter(cField -> !Modifier.isTransient(cField.getModifiers())); 56 | } 57 | 58 | public Map blindMap() { 59 | Map toRet = new LinkedHashMap<>(); 60 | rowNames.entrySet().stream() 61 | .sorted(Comparator.comparing(Map.Entry::getValue)) 62 | .forEach(entry -> toRet.put(entry.getKey(), super.getField(entry.getValue()))); 63 | return toRet; 64 | } 65 | 66 | @Override 67 | public String toString() { 68 | return blindMap().toString(); 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/src/main/java/com/zhisheng/connectors/kudu/connector/KuduRowIterator.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kudu.connector; 2 | 3 | import org.apache.kudu.client.KuduException; 4 | import org.apache.kudu.client.KuduScanner; 5 | import org.apache.kudu.client.RowResult; 6 | import org.apache.kudu.client.RowResultIterator; 7 | 8 | /** 9 | * Desc: 10 | * Created by zhisheng on 2019-06-08 11 | * blog:http://www.54tianzhisheng.cn/ 12 | * 微信公众号:zhisheng 13 | */ 14 | public class KuduRowIterator { 15 | private KuduScanner scanner; 16 | private RowResultIterator rowIterator; 17 | 18 | public KuduRowIterator(KuduScanner scanner) throws KuduException { 19 | this.scanner = scanner; 20 | nextRows(); 21 | } 22 | 23 | public void close() throws KuduException { 24 | scanner.close(); 25 | } 26 | 27 | public boolean hasNext() throws KuduException { 28 | if (rowIterator.hasNext()) { 29 | return true; 30 | } else if (scanner.hasMoreRows()) { 31 | nextRows(); 32 | return true; 33 | } else { 34 | return false; 35 | } 36 | } 37 | 38 | public KuduRow next() { 39 | RowResult row = this.rowIterator.next(); 40 | return KuduMapper.toKuduRow(row); 41 | } 42 | 43 | private void nextRows() throws KuduException { 44 | this.rowIterator = scanner.nextRows(); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/src/main/java/com/zhisheng/connectors/kudu/serde/DefaultSerDe.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kudu.serde; 2 | 3 | import com.zhisheng.connectors.kudu.connector.KuduRow; 4 | import org.apache.kudu.Schema; 5 | /** 6 | * Desc: 7 | * Created by zhisheng on 2019-06-08 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public class DefaultSerDe implements KuduSerialization, KuduDeserialization { 12 | 13 | @Override 14 | public KuduRow deserialize(KuduRow row) { 15 | return row; 16 | } 17 | 18 | @Override 19 | public KuduRow serialize(KuduRow value) { 20 | return value; 21 | } 22 | 23 | @Override 24 | public DefaultSerDe withSchema(Schema schema) { 25 | return this; 26 | } 27 | 28 | } 29 | 30 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/src/main/java/com/zhisheng/connectors/kudu/serde/KuduDeserialization.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kudu.serde; 2 | 3 | import com.zhisheng.connectors.kudu.connector.KuduRow; 4 | 5 | import java.io.Serializable; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-08 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public interface KuduDeserialization extends Serializable { 14 | T deserialize(KuduRow row); 15 | } -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-kudu/src/main/java/com/zhisheng/connectors/kudu/serde/KuduSerialization.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.kudu.serde; 2 | 3 | import com.zhisheng.connectors.kudu.connector.KuduRow; 4 | import org.apache.kudu.Schema; 5 | 6 | import java.io.Serializable; 7 | 8 | /** 9 | * Desc: 10 | * Created by zhisheng on 2019-06-08 11 | * blog:http://www.54tianzhisheng.cn/ 12 | * 微信公众号:zhisheng 13 | */ 14 | public interface KuduSerialization extends Serializable { 15 | KuduRow serialize(T value); 16 | 17 | KuduSerialization withSchema(Schema schema); 18 | } -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/README.md: -------------------------------------------------------------------------------- 1 | ## Flink connector MySQL 2 | 3 | [http://www.54tianzhisheng.cn/2019/01/15/Flink-MySQL-sink/](http://www.54tianzhisheng.cn/2019/01/15/Flink-MySQL-sink/) -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/java/com/zhisheng/connectors/mysql/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.mysql; 2 | 3 | 4 | import com.google.common.collect.Lists; 5 | import com.zhisheng.common.utils.ExecutionEnvUtil; 6 | import com.zhisheng.common.utils.GsonUtil; 7 | import com.zhisheng.common.utils.KafkaConfigUtil; 8 | import com.zhisheng.connectors.mysql.model.Student; 9 | import com.zhisheng.connectors.mysql.sinks.SinkToMySQL; 10 | import lombok.extern.slf4j.Slf4j; 11 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 12 | import org.apache.flink.api.java.utils.ParameterTool; 13 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 14 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 15 | import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; 16 | import org.apache.flink.streaming.api.windowing.time.Time; 17 | import org.apache.flink.streaming.api.windowing.windows.TimeWindow; 18 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; 19 | import org.apache.flink.util.Collector; 20 | 21 | import java.util.ArrayList; 22 | import java.util.List; 23 | import java.util.Properties; 24 | 25 | import static com.zhisheng.common.constant.PropertiesConstants.*; 26 | 27 | /** 28 | * Created by zhisheng on 2019-02-17 29 | * Blog: http://www.54tianzhisheng.cn/2019/01/09/Flink-MySQL-sink/ 30 | */ 31 | @Slf4j 32 | public class Main { 33 | public static void main(String[] args) throws Exception { 34 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 35 | ParameterTool parameterTool = ExecutionEnvUtil.PARAMETER_TOOL; 36 | Properties props = KafkaConfigUtil.buildKafkaProps(parameterTool); 37 | 38 | SingleOutputStreamOperator student = env.addSource(new FlinkKafkaConsumer011<>( 39 | parameterTool.get(METRICS_TOPIC), //这个 kafka topic 需要和上面的工具类的 topic 一致 40 | new SimpleStringSchema(), 41 | props)).setParallelism(parameterTool.getInt(STREAM_PARALLELISM, 1)) 42 | .map(string -> GsonUtil.fromJson(string, Student.class)).setParallelism(4); //解析字符串成 student 对象 43 | 44 | //timeWindowAll 并行度只能为 1 45 | student.timeWindowAll(Time.minutes(1)).apply(new AllWindowFunction, TimeWindow>() { 46 | @Override 47 | public void apply(TimeWindow window, Iterable values, Collector> out) throws Exception { 48 | ArrayList students = Lists.newArrayList(values); 49 | if (students.size() > 0) { 50 | log.info("1 分钟内收集到 student 的数据条数是:" + students.size()); 51 | out.collect(students); 52 | } 53 | } 54 | }).addSink(new SinkToMySQL()).setParallelism(parameterTool.getInt(STREAM_SINK_PARALLELISM, 1)); 55 | 56 | env.execute("flink learning connectors mysql"); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/java/com/zhisheng/connectors/mysql/model/Student.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.mysql.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-02-17 10 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 11 | */ 12 | 13 | @Data 14 | @AllArgsConstructor 15 | @NoArgsConstructor 16 | public class Student { 17 | public int id; 18 | public String name; 19 | public String password; 20 | public int age; 21 | } 22 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/java/com/zhisheng/connectors/mysql/sinks/SinkToMySQL.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.mysql.sinks; 2 | 3 | import com.zhisheng.connectors.mysql.model.Student; 4 | import org.apache.commons.dbcp2.BasicDataSource; 5 | import org.apache.flink.configuration.Configuration; 6 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 7 | 8 | import javax.sql.DataSource; 9 | import java.sql.Connection; 10 | import java.sql.DriverManager; 11 | import java.sql.PreparedStatement; 12 | import java.util.List; 13 | 14 | /** 15 | * Desc: 数据批量 sink 数据到 mysql 16 | * Created by zhisheng_tian on 2019-02-17 17 | * Blog: http://www.54tianzhisheng.cn/2019/01/09/Flink-MySQL-sink/ 18 | */ 19 | public class SinkToMySQL extends RichSinkFunction> { 20 | PreparedStatement ps; 21 | BasicDataSource dataSource; 22 | private Connection connection; 23 | 24 | /** 25 | * open() 方法中建立连接,这样不用每次 invoke 的时候都要建立连接和释放连接 26 | * 27 | * @param parameters 28 | * @throws Exception 29 | */ 30 | @Override 31 | public void open(Configuration parameters) throws Exception { 32 | super.open(parameters); 33 | dataSource = new BasicDataSource(); 34 | connection = getConnection(dataSource); 35 | String sql = "insert into Student(id, name, password, age) values(?, ?, ?, ?);"; 36 | ps = this.connection.prepareStatement(sql); 37 | } 38 | 39 | @Override 40 | public void close() throws Exception { 41 | super.close(); 42 | //关闭连接和释放资源 43 | if (connection != null) { 44 | connection.close(); 45 | } 46 | if (ps != null) { 47 | ps.close(); 48 | } 49 | } 50 | 51 | /** 52 | * 每条数据的插入都要调用一次 invoke() 方法 53 | * 54 | * @param value 55 | * @param context 56 | * @throws Exception 57 | */ 58 | @Override 59 | public void invoke(List value, Context context) throws Exception { 60 | //遍历数据集合 61 | for (Student student : value) { 62 | ps.setInt(1, student.getId()); 63 | ps.setString(2, student.getName()); 64 | ps.setString(3, student.getPassword()); 65 | ps.setInt(4, student.getAge()); 66 | ps.addBatch(); 67 | } 68 | int[] count = ps.executeBatch();//批量后执行 69 | System.out.println("成功了插入了" + count.length + "行数据"); 70 | } 71 | 72 | 73 | private static Connection getConnection(BasicDataSource dataSource) { 74 | dataSource.setDriverClassName("com.mysql.jdbc.Driver"); 75 | //注意,替换成自己本地的 mysql 数据库地址和用户名、密码 76 | dataSource.setUrl("jdbc:mysql://localhost:3306/test"); 77 | dataSource.setUsername("root"); 78 | dataSource.setPassword("root123456"); 79 | //设置连接池的一些参数 80 | dataSource.setInitialSize(10); 81 | dataSource.setMaxTotal(50); 82 | dataSource.setMinIdle(2); 83 | 84 | Connection con = null; 85 | try { 86 | con = dataSource.getConnection(); 87 | System.out.println("创建连接池:" + con); 88 | } catch (Exception e) { 89 | System.out.println("-----------mysql get connection has exception , msg = " + e.getMessage()); 90 | } 91 | return con; 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/java/com/zhisheng/connectors/mysql/utils/KafkaUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.mysql.utils; 2 | 3 | import com.zhisheng.common.utils.GsonUtil; 4 | import com.zhisheng.connectors.mysql.model.Student; 5 | import org.apache.kafka.clients.producer.KafkaProducer; 6 | import org.apache.kafka.clients.producer.ProducerRecord; 7 | 8 | import java.util.Properties; 9 | 10 | /** 11 | * Desc: 往kafka中写数据,可以使用这个main函数进行测试 12 | * Created by zhisheng on 2019-02-17 13 | * Blog: http://www.54tianzhisheng.cn/2019/01/09/Flink-MySQL-sink/ 14 | */ 15 | public class KafkaUtil { 16 | public static final String broker_list = "localhost:9092"; 17 | public static final String topic = "student"; //kafka topic 需要和 flink 程序用同一个 topic 18 | 19 | public static void writeToKafka() throws InterruptedException { 20 | Properties props = new Properties(); 21 | props.put("bootstrap.servers", broker_list); 22 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 23 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 24 | KafkaProducer producer = new KafkaProducer(props); 25 | 26 | for (int i = 1; i <= 100; i++) { 27 | Student student = new Student(i, "zhisheng" + i, "password" + i, 18 + i); 28 | ProducerRecord record = new ProducerRecord(topic, null, null, GsonUtil.toJson(student)); 29 | producer.send(record); 30 | System.out.println("发送数据: " + GsonUtil.toJson(student)); 31 | Thread.sleep(10 * 1000); //发送一条数据 sleep 10s,相当于 1 分钟 6 条 32 | } 33 | producer.flush(); 34 | } 35 | 36 | public static void main(String[] args) throws InterruptedException { 37 | writeToKafka(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metric-group 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=student 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-mysql/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-netty/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-netty 13 | 14 | 15 | 16 | io.netty 17 | netty-all 18 | 4.1.6.Final 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | org.apache.maven.plugins 27 | maven-shade-plugin 28 | 3.1.0 29 | 30 | false 31 | 32 | 33 | 34 | package 35 | 36 | shade 37 | 38 | 39 | 40 | 41 | 42 | 44 | com.zhisheng.connectors.netty.Main 45 | 46 | 48 | reference.conf 49 | 50 | 51 | 52 | 53 | *:*:*:* 54 | 55 | META-INF/*.SF 56 | META-INF/*.DSA 57 | META-INF/*.RSA 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-netty/src/main/java/com/zhisheng/connectors/netty/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.netty; 2 | 3 | import com.zhisheng.common.utils.ExecutionEnvUtil; 4 | import org.apache.flink.api.java.utils.ParameterTool; 5 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 6 | 7 | /** 8 | * Desc: Netty connector 9 | * Created by zhisheng on 2019-05-04 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class Main { 14 | public static void main(String[] args) throws Exception { 15 | ParameterTool parameterTool = ExecutionEnvUtil.PARAMETER_TOOL; 16 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 17 | 18 | 19 | 20 | env.execute("flink netty connector"); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/README.md: -------------------------------------------------------------------------------- 1 | ## Flink connector RabbitMQ 2 | 3 | [http://www.54tianzhisheng.cn/2019/01/20/Flink-RabbitMQ-sink/](http://www.54tianzhisheng.cn/2019/01/20/Flink-RabbitMQ-sink/) -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-rabbitmq 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-connector-rabbitmq_${scala.binary.version} 18 | ${flink.version} 19 | 20 | 21 | 22 | 23 | 24 | 25 | org.apache.maven.plugins 26 | maven-shade-plugin 27 | 3.1.0 28 | 29 | false 30 | 31 | 32 | 33 | package 34 | 35 | shade 36 | 37 | 38 | 39 | 40 | 41 | 43 | com.zhisheng.connectors.rabbitmq.Main 44 | 45 | 47 | reference.conf 48 | 49 | 50 | 51 | 52 | *:*:*:* 53 | 54 | META-INF/*.SF 55 | META-INF/*.DSA 56 | META-INF/*.RSA 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/java/com/zhisheng/connectors/rabbitmq/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rabbitmq; 2 | 3 | 4 | import com.zhisheng.common.utils.ExecutionEnvUtil; 5 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 6 | import org.apache.flink.api.java.utils.ParameterTool; 7 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 8 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 9 | import org.apache.flink.streaming.connectors.rabbitmq.RMQSource; 10 | import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig; 11 | 12 | 13 | /** 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | * 从 rabbitmq 读取数据 17 | */ 18 | public class Main { 19 | public static void main(String[] args) throws Exception { 20 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 21 | ParameterTool parameterTool = ExecutionEnvUtil.PARAMETER_TOOL; 22 | 23 | //下面这些写死的参数可以放在配置文件中,然后通过 parameterTool 获取 24 | final RMQConnectionConfig connectionConfig = new RMQConnectionConfig 25 | .Builder().setHost("localhost").setVirtualHost("/") 26 | .setPort(5672).setUserName("admin").setPassword("admin") 27 | .build(); 28 | 29 | DataStreamSource zhisheng = env.addSource(new RMQSource<>(connectionConfig, 30 | "zhisheng", 31 | true, 32 | new SimpleStringSchema())) 33 | .setParallelism(1); 34 | zhisheng.print(); 35 | 36 | //如果想保证 exactly-once 或 at-least-once 需要把 checkpoint 开启 37 | // env.enableCheckpointing(10000); 38 | env.execute("flink learning connectors rabbitmq"); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/java/com/zhisheng/connectors/rabbitmq/Main1.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rabbitmq; 2 | 3 | 4 | import com.zhisheng.common.model.MetricEvent; 5 | import com.zhisheng.common.schemas.MetricSchema; 6 | import com.zhisheng.common.utils.ExecutionEnvUtil; 7 | import com.zhisheng.common.utils.KafkaConfigUtil; 8 | import org.apache.flink.api.java.utils.ParameterTool; 9 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 10 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 11 | import org.apache.flink.streaming.connectors.rabbitmq.RMQSink; 12 | import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig; 13 | 14 | 15 | /** 16 | * blog:http://www.54tianzhisheng.cn/ 17 | * 微信公众号:zhisheng 18 | * 从 kafka 读取数据 sink 到 rabbitmq 19 | */ 20 | public class Main1 { 21 | public static void main(String[] args) throws Exception { 22 | final ParameterTool parameterTool = ExecutionEnvUtil.createParameterTool(args); 23 | StreamExecutionEnvironment env = ExecutionEnvUtil.prepare(parameterTool); 24 | DataStreamSource data = KafkaConfigUtil.buildSource(env); 25 | 26 | final RMQConnectionConfig connectionConfig = new RMQConnectionConfig 27 | .Builder().setHost("localhost").setVirtualHost("/") 28 | .setPort(5672).setUserName("admin").setPassword("admin") 29 | .build(); 30 | 31 | //注意,换一个新的 queue,否则也会报错 32 | data.addSink(new RMQSink<>(connectionConfig, "zhisheng001", new MetricSchema())); 33 | env.execute("flink learning connectors rabbitmq"); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/java/com/zhisheng/connectors/rabbitmq/model/EndPoint.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rabbitmq.model; 2 | 3 | import com.rabbitmq.client.Channel; 4 | import com.rabbitmq.client.Connection; 5 | import com.rabbitmq.client.ConnectionFactory; 6 | 7 | import java.io.IOException; 8 | 9 | /** 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class EndPoint { 14 | protected Channel channel; 15 | protected Connection connection; 16 | protected String endPointName; 17 | 18 | public EndPoint(String endpointName) throws Exception { 19 | this.endPointName = endpointName; 20 | 21 | ConnectionFactory factory = new ConnectionFactory(); 22 | 23 | factory.setHost("127.0.0.1"); 24 | factory.setUsername("admin"); 25 | factory.setPassword("admin"); 26 | factory.setPort(5672); 27 | 28 | connection = factory.newConnection(); 29 | 30 | 31 | channel = connection.createChannel(); 32 | 33 | channel.queueDeclare(endpointName, false, false, false, null); 34 | } 35 | 36 | /** 37 | * 关闭channel和connection。并非必须,因为隐含是自动调用的 38 | * 39 | * @throws IOException 40 | */ 41 | public void close() throws Exception { 42 | this.channel.close(); 43 | this.connection.close(); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/java/com/zhisheng/connectors/rabbitmq/utils/RabbitMQProducerUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rabbitmq.utils; 2 | 3 | 4 | import com.rabbitmq.client.Channel; 5 | import com.rabbitmq.client.Connection; 6 | import com.rabbitmq.client.ConnectionFactory; 7 | 8 | /** 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | public class RabbitMQProducerUtil { 13 | public final static String QUEUE_NAME = "zhisheng"; 14 | 15 | public static void main(String[] args) throws Exception { 16 | //创建连接工厂 17 | ConnectionFactory factory = new ConnectionFactory(); 18 | 19 | //设置RabbitMQ相关信息 20 | factory.setHost("localhost"); 21 | factory.setUsername("admin"); 22 | factory.setPassword("admin"); 23 | factory.setPort(5672); 24 | 25 | //创建一个新的连接 26 | Connection connection = factory.newConnection(); 27 | 28 | //创建一个通道 29 | Channel channel = connection.createChannel(); 30 | 31 | // 声明一个队列 32 | // channel.queueDeclare(QUEUE_NAME, false, false, false, null); 33 | 34 | //发送消息到队列中 35 | String message = "Hello zhisheng"; 36 | 37 | for (int i = 0; i < 1000; i++) { 38 | channel.basicPublish("", QUEUE_NAME, null, (message + i).getBytes("UTF-8")); 39 | System.out.println("Producer Send +'" + message + i); 40 | } 41 | 42 | //关闭通道和连接 43 | channel.close(); 44 | connection.close(); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=student 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false 10 | rmq.host=localhost 11 | rmq.port=5672 12 | rmq.user=admin 13 | rmq.password=admin -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rabbitmq/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/README.md: -------------------------------------------------------------------------------- 1 | ### Flink connector Redis 2 | 3 | 利用自带的 Redis Connector 从 Kafka 中读取数据,然后写入到 Redis。 4 | 5 | Redis 分三种情况: 6 | 7 | + 单机 Redis 8 | 9 | + Redis 集群 10 | 11 | + Redis Sentinels -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-connectors 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors-redis 13 | 14 | 15 | 16 | org.apache.flink 17 | flink-connector-redis_2.10 18 | 1.1.5 19 | 20 | 21 | 22 | redis.clients 23 | jedis 24 | 2.9.0 25 | 26 | 27 | 28 | 29 | 30 | 31 | org.apache.maven.plugins 32 | maven-shade-plugin 33 | 3.1.0 34 | 35 | false 36 | 37 | 38 | 39 | package 40 | 41 | shade 42 | 43 | 44 | 45 | 46 | 47 | 49 | com.zhisheng.connectors.redis.Main 50 | 51 | 53 | reference.conf 54 | 55 | 56 | 57 | 58 | *:*:*:* 59 | 60 | META-INF/*.SF 61 | META-INF/*.DSA 62 | META-INF/*.RSA 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/src/main/java/com/zhisheng/connectors/redis/utils/ProductUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.redis.utils; 2 | 3 | import com.zhisheng.common.model.ProductEvent; 4 | import com.zhisheng.common.utils.GsonUtil; 5 | import org.apache.kafka.clients.producer.KafkaProducer; 6 | import org.apache.kafka.clients.producer.ProducerRecord; 7 | 8 | import java.util.Properties; 9 | import java.util.Random; 10 | 11 | /** 12 | * Desc: 13 | * Created by zhisheng on 2019-04-29 14 | * blog:http://www.54tianzhisheng.cn/ 15 | * 微信公众号:zhisheng 16 | */ 17 | public class ProductUtil { 18 | public static final String broker_list = "localhost:9092"; 19 | public static final String topic = "zhisheng"; //kafka topic 需要和 flink 程序用同一个 topic 20 | 21 | public static final Random random = new Random(); 22 | 23 | public static void main(String[] args) { 24 | Properties props = new Properties(); 25 | props.put("bootstrap.servers", broker_list); 26 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 27 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 28 | KafkaProducer producer = new KafkaProducer(props); 29 | 30 | for (int i = 1; i <= 10000; i++) { 31 | ProductEvent product = ProductEvent.builder().id((long) i) //商品的 id 32 | .name("product" + i) //商品 name 33 | .price(random.nextLong() / 10000000000000L) //商品价格(以分为单位) 34 | .code("code" + i).build(); //商品编码 35 | 36 | ProducerRecord record = new ProducerRecord(topic, null, null, GsonUtil.toJson(product)); 37 | producer.send(record); 38 | System.out.println("发送数据: " + GsonUtil.toJson(product)); 39 | } 40 | producer.flush(); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false 10 | redis.host=127.0.0.1 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-redis/src/test/java/RedisTest.java: -------------------------------------------------------------------------------- 1 | import redis.clients.jedis.Jedis; 2 | 3 | /** 4 | * Desc: 验证数据已经写入到 Redis 5 | * Created by zhisheng on 2019-04-29 6 | * blog:http://www.54tianzhisheng.cn/ 7 | * 微信公众号:zhisheng 8 | */ 9 | public class RedisTest { 10 | public static void main(String[] args) { 11 | Jedis jedis = new Jedis("127.0.0.1"); 12 | System.out.println("Server is running: " + jedis.ping()); 13 | System.out.println("result:" + jedis.hgetAll("zhisheng")); 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/README.md: -------------------------------------------------------------------------------- 1 | ### Flink-learning-connectors-rocketmq 2 | 3 | Flink 消费 RocketMQ 数据,转换后再将转换后到数据发送到 RocketMQ,demo 类可以参考 RocketMQFlinkExample 类。 -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/RocketMQUtils.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq; 2 | 3 | import java.util.Properties; 4 | 5 | /** 6 | * Desc: RocketMQ 工具类 7 | * Created by zhisheng on 2019-06-05 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public final class RocketMQUtils { 12 | 13 | public static int getInteger(Properties props, String key, int defaultValue) { 14 | return Integer.parseInt(props.getProperty(key, String.valueOf(defaultValue))); 15 | } 16 | 17 | public static long getLong(Properties props, String key, long defaultValue) { 18 | return Long.parseLong(props.getProperty(key, String.valueOf(defaultValue))); 19 | } 20 | 21 | public static boolean getBoolean(Properties props, String key, boolean defaultValue) { 22 | return Boolean.parseBoolean(props.getProperty(key, String.valueOf(defaultValue))); 23 | } 24 | 25 | } 26 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/RunningChecker.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * Desc: 7 | * Created by zhisheng on 2019-06-05 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public class RunningChecker implements Serializable { 12 | private volatile boolean isRunning = false; 13 | 14 | public boolean isRunning() { 15 | return isRunning; 16 | } 17 | 18 | public void setRunning(boolean running) { 19 | isRunning = running; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/selector/DefaultTopicSelector.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.selector; 2 | 3 | /** 4 | * Desc: 5 | * Created by zhisheng on 2019-06-05 6 | * blog:http://www.54tianzhisheng.cn/ 7 | * 微信公众号:zhisheng 8 | */ 9 | public class DefaultTopicSelector implements TopicSelector { 10 | 11 | private final String topicName; 12 | private final String tagName; 13 | 14 | public DefaultTopicSelector(final String topicName) { 15 | this(topicName, ""); 16 | } 17 | 18 | public DefaultTopicSelector(String topicName, String tagName) { 19 | this.topicName = topicName; 20 | this.tagName = tagName; 21 | } 22 | 23 | @Override 24 | public String getTopic(T tuple) { 25 | return topicName; 26 | } 27 | 28 | @Override 29 | public String getTag(T tuple) { 30 | return tagName; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/selector/SimpleTopicSelector.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.selector; 2 | 3 | import org.slf4j.Logger; 4 | import org.slf4j.LoggerFactory; 5 | 6 | import java.util.Map; 7 | 8 | /** 9 | * Desc: 10 | * Created by zhisheng on 2019-06-05 11 | * blog:http://www.54tianzhisheng.cn/ 12 | * 微信公众号:zhisheng 13 | */ 14 | public class SimpleTopicSelector implements TopicSelector { 15 | 16 | private static final Logger LOG = LoggerFactory.getLogger(SimpleTopicSelector.class); 17 | 18 | private final String topicFieldName; 19 | private final String defaultTopicName; 20 | 21 | private final String tagFieldName; 22 | private final String defaultTagName; 23 | 24 | 25 | public SimpleTopicSelector(String topicFieldName, String defaultTopicName, String tagFieldName, String defaultTagName) { 26 | this.topicFieldName = topicFieldName; 27 | this.defaultTopicName = defaultTopicName; 28 | this.tagFieldName = tagFieldName; 29 | this.defaultTagName = defaultTagName; 30 | } 31 | 32 | @Override 33 | public String getTopic(Map tuple) { 34 | if (tuple.containsKey(topicFieldName)) { 35 | Object topic = tuple.get(topicFieldName); 36 | return topic != null ? topic.toString() : defaultTopicName; 37 | } else { 38 | LOG.warn("Field {} Not Found. Returning default topic {}", topicFieldName, defaultTopicName); 39 | return defaultTopicName; 40 | } 41 | } 42 | 43 | @Override 44 | public String getTag(Map tuple) { 45 | if (tuple.containsKey(tagFieldName)) { 46 | Object tag = tuple.get(tagFieldName); 47 | return tag != null ? tag.toString() : defaultTagName; 48 | } else { 49 | LOG.warn("Field {} Not Found. Returning default tag {}", tagFieldName, defaultTagName); 50 | return defaultTagName; 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/selector/TopicSelector.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.selector; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * Desc: 7 | * Created by zhisheng on 2019-06-05 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public interface TopicSelector extends Serializable { 12 | 13 | String getTopic(T tuple); 14 | 15 | String getTag(T tuple); 16 | 17 | } -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/serialization/KeyValueDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.serialization; 2 | 3 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 4 | 5 | import java.io.Serializable; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-05 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public interface KeyValueDeserializationSchema extends ResultTypeQueryable, Serializable { 14 | T deserializeKeyAndValue(byte[] key, byte[] value); 15 | } 16 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/serialization/KeyValueSerializationSchema.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.serialization; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * Desc: 7 | * Created by zhisheng on 2019-06-05 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public interface KeyValueSerializationSchema extends Serializable { 12 | byte[] serializeKey(T tuple); 13 | 14 | byte[] serializeValue(T tuple); 15 | } 16 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/serialization/SimpleKeyValueDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.serialization; 2 | 3 | import org.apache.flink.api.common.typeinfo.TypeInformation; 4 | 5 | import java.nio.charset.StandardCharsets; 6 | import java.util.HashMap; 7 | import java.util.Map; 8 | 9 | /** 10 | * Desc: 11 | * Created by zhisheng on 2019-06-05 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class SimpleKeyValueDeserializationSchema implements KeyValueDeserializationSchema { 16 | 17 | public static final String DEFAULT_KEY_FIELD = "key"; 18 | public static final String DEFAULT_VALUE_FIELD = "value"; 19 | 20 | public String keyField; 21 | public String valueField; 22 | 23 | public SimpleKeyValueDeserializationSchema() { 24 | this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD); 25 | } 26 | 27 | public SimpleKeyValueDeserializationSchema(String keyField, String valueField) { 28 | this.keyField = keyField; 29 | this.valueField = valueField; 30 | } 31 | 32 | @Override 33 | public Map deserializeKeyAndValue(byte[] key, byte[] value) { 34 | HashMap map = new HashMap(2); 35 | if (keyField != null) { 36 | String k = key != null ? new String(key, StandardCharsets.UTF_8) : null; 37 | map.put(keyField, k); 38 | } 39 | 40 | if (valueField != null) { 41 | String v = value != null ? new String(value, StandardCharsets.UTF_8) : null; 42 | map.put(valueField, v); 43 | } 44 | return map; 45 | } 46 | 47 | @Override 48 | public TypeInformation getProducedType() { 49 | return null; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/common/serialization/SimpleKeyValueSerializationSchema.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.common.serialization; 2 | 3 | import java.nio.charset.StandardCharsets; 4 | import java.util.Map; 5 | 6 | /** 7 | * Desc: 8 | * Created by zhisheng on 2019-06-05 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | public class SimpleKeyValueSerializationSchema implements KeyValueSerializationSchema { 13 | 14 | public static final String DEFAULT_KEY_FIELD = "key"; 15 | public static final String DEFAULT_VALUE_FIELD = "value"; 16 | 17 | public String keyField; 18 | public String valueField; 19 | 20 | public SimpleKeyValueSerializationSchema() { 21 | this(DEFAULT_KEY_FIELD, DEFAULT_VALUE_FIELD); 22 | } 23 | 24 | public SimpleKeyValueSerializationSchema(String keyField, String valueField) { 25 | this.keyField = keyField; 26 | this.valueField = valueField; 27 | } 28 | 29 | @Override 30 | public byte[] serializeKey(Map tuple) { 31 | return getBytes(tuple, keyField); 32 | } 33 | 34 | @Override 35 | public byte[] serializeValue(Map tuple) { 36 | return getBytes(tuple, valueField); 37 | } 38 | 39 | private byte[] getBytes(Map tuple, String key) { 40 | if (tuple == null || key == null) { 41 | return null; 42 | } 43 | Object value = tuple.get(key); 44 | return value != null ? value.toString().getBytes(StandardCharsets.UTF_8) : null; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/example/SimpleConsumer.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.example; 2 | 3 | import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; 4 | import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; 5 | import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; 6 | import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; 7 | import org.apache.rocketmq.client.exception.MQClientException; 8 | import org.apache.rocketmq.common.message.MessageExt; 9 | 10 | import java.util.List; 11 | 12 | /** 13 | * Desc: 14 | * Created by zhisheng on 2019-06-07 15 | * blog:http://www.54tianzhisheng.cn/ 16 | * 微信公众号:zhisheng 17 | */ 18 | public class SimpleConsumer { 19 | public static void main(String[] args) { 20 | DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("g00003"); 21 | consumer.setNamesrvAddr("localhost:9876"); 22 | try { 23 | consumer.subscribe("zhisheng", "*"); 24 | } catch (MQClientException e) { 25 | e.printStackTrace(); 26 | } 27 | consumer.registerMessageListener(new MessageListenerConcurrently() { 28 | @Override 29 | public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { 30 | for (MessageExt msg : msgs) { 31 | System.out.println(msg.getKeys() + ":" + new String(msg.getBody())); 32 | } 33 | return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; 34 | } 35 | }); 36 | try { 37 | consumer.start(); 38 | } catch (MQClientException e) { 39 | e.printStackTrace(); 40 | } 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/java/com/zhisheng/connectors/rocketmq/example/SimpleProducer.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.connectors.rocketmq.example; 2 | 3 | import org.apache.rocketmq.client.exception.MQClientException; 4 | import org.apache.rocketmq.client.producer.DefaultMQProducer; 5 | import org.apache.rocketmq.common.message.Message; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-07 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | public class SimpleProducer { 14 | public static void main(String[] args) { 15 | DefaultMQProducer producer = new DefaultMQProducer("p001"); 16 | producer.setNamesrvAddr("localhost:9876"); 17 | try { 18 | producer.start(); 19 | } catch (MQClientException e) { 20 | e.printStackTrace(); 21 | } 22 | for (int i = 0; i < 10000; i++) { 23 | Message msg = new Message("zhisheng", "", "id_" + i, ("country_X province_" + i).getBytes()); 24 | try { 25 | producer.send(msg); 26 | } catch (Exception e) { 27 | e.printStackTrace(); 28 | } 29 | System.out.println("send " + i); 30 | try { 31 | Thread.sleep(10); 32 | } catch (InterruptedException e) { 33 | e.printStackTrace(); 34 | } 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-connectors/flink-learning-connectors-rocketmq/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-connectors/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-connectors 13 | pom 14 | 15 | flink-learning-connectors-es6 16 | flink-learning-connectors-kafka 17 | flink-learning-connectors-cassandra 18 | flink-learning-connectors-mysql 19 | flink-learning-connectors-rabbitmq 20 | flink-learning-connectors-hdfs 21 | flink-learning-connectors-hbase 22 | flink-learning-connectors-rocketmq 23 | flink-learning-connectors-redis 24 | flink-learning-connectors-flume 25 | flink-learning-connectors-influxdb 26 | flink-learning-connectors-activemq 27 | flink-learning-connectors-netty 28 | flink-learning-connectors-akka 29 | flink-learning-connectors-kudu 30 | 31 | 32 | 33 | 34 | com.zhisheng.flink 35 | flink-learning-common 36 | 1.0-SNAPSHOT 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/README.md: -------------------------------------------------------------------------------- 1 | ## Flink data sink 2 | 3 | [http://www.54tianzhisheng.cn/2018/10/29/flink-sink/](http://www.54tianzhisheng.cn/2018/10/29/flink-sink/) 4 | 5 | [http://www.54tianzhisheng.cn/2018/10/31/flink-create-sink/](http://www.54tianzhisheng.cn/2018/10/31/flink-create-sink/) -------------------------------------------------------------------------------- /flink-learning-data-sinks/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-data-sinks 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | mysql 22 | mysql-connector-java 23 | 5.1.34 24 | 25 | 26 | 27 | 28 | 29 | 30 | org.apache.maven.plugins 31 | maven-shade-plugin 32 | 3.1.0 33 | 34 | false 35 | 36 | 37 | 38 | package 39 | 40 | shade 41 | 42 | 43 | 44 | 45 | 46 | 48 | 49 | com.zhisheng.data.sinks.Main 50 | 51 | 53 | reference.conf 54 | 55 | 56 | 57 | 58 | *:*:*:* 59 | 60 | META-INF/*.SF 61 | META-INF/*.DSA 62 | META-INF/*.RSA 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/java/com/zhisheng/data/sinks/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sinks; 2 | 3 | 4 | import com.zhisheng.common.utils.ExecutionEnvUtil; 5 | import com.zhisheng.common.utils.GsonUtil; 6 | import com.zhisheng.common.utils.KafkaConfigUtil; 7 | import com.zhisheng.data.sinks.model.Student; 8 | import com.zhisheng.data.sinks.sinks.SinkToMySQL; 9 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 10 | import org.apache.flink.api.java.utils.ParameterTool; 11 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 12 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 13 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; 14 | 15 | import java.util.Properties; 16 | 17 | import static com.zhisheng.common.constant.PropertiesConstants.METRICS_TOPIC; 18 | 19 | /** 20 | * blog:http://www.54tianzhisheng.cn/ 21 | * 微信公众号:zhisheng 22 | */ 23 | public class Main { 24 | public static void main(String[] args) throws Exception{ 25 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 26 | ParameterTool parameterTool = ExecutionEnvUtil.PARAMETER_TOOL; 27 | Properties props = KafkaConfigUtil.buildKafkaProps(parameterTool); 28 | 29 | SingleOutputStreamOperator student = env.addSource(new FlinkKafkaConsumer011<>( 30 | parameterTool.get(METRICS_TOPIC), //这个 kafka topic 需要和上面的工具类的 topic 一致 31 | new SimpleStringSchema(), 32 | props)).setParallelism(1) 33 | .map(string -> GsonUtil.fromJson(string, Student.class)); //博客里面用的是 fastjson,这里用的是gson解析,解析字符串成 student 对象 34 | 35 | student.addSink(new SinkToMySQL()); //数据 sink 到 mysql 36 | 37 | env.execute("Flink data sink"); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/java/com/zhisheng/data/sinks/model/Student.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sinks.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-02-17 10 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 11 | */ 12 | 13 | @Data 14 | @AllArgsConstructor 15 | @NoArgsConstructor 16 | public class Student { 17 | public int id; 18 | public String name; 19 | public String password; 20 | public int age; 21 | } 22 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/java/com/zhisheng/data/sinks/sinks/SinkToMySQL.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sinks.sinks; 2 | 3 | import com.zhisheng.data.sinks.model.Student; 4 | import org.apache.flink.configuration.Configuration; 5 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 6 | 7 | import java.sql.Connection; 8 | import java.sql.DriverManager; 9 | import java.sql.PreparedStatement; 10 | 11 | /** 12 | * Desc: sink 数据到 mysql 13 | * Created by zhisheng_tian on 2019-02-17 14 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 15 | */ 16 | public class SinkToMySQL extends RichSinkFunction { 17 | PreparedStatement ps; 18 | private Connection connection; 19 | 20 | /** 21 | * open() 方法中建立连接,这样不用每次 invoke 的时候都要建立连接和释放连接 22 | * 23 | * @param parameters 24 | * @throws Exception 25 | */ 26 | @Override 27 | public void open(Configuration parameters) throws Exception { 28 | super.open(parameters); 29 | connection = getConnection(); 30 | String sql = "insert into Student(id, name, password, age) values(?, ?, ?, ?);"; 31 | ps = this.connection.prepareStatement(sql); 32 | } 33 | 34 | @Override 35 | public void close() throws Exception { 36 | super.close(); 37 | //关闭连接和释放资源 38 | if (connection != null) { 39 | connection.close(); 40 | } 41 | if (ps != null) { 42 | ps.close(); 43 | } 44 | } 45 | 46 | /** 47 | * 每条数据的插入都要调用一次 invoke() 方法 48 | * 49 | * @param value 50 | * @param context 51 | * @throws Exception 52 | */ 53 | @Override 54 | public void invoke(Student value, Context context) throws Exception { 55 | //组装数据,执行插入操作 56 | ps.setInt(1, value.getId()); 57 | ps.setString(2, value.getName()); 58 | ps.setString(3, value.getPassword()); 59 | ps.setInt(4, value.getAge()); 60 | ps.executeUpdate(); 61 | } 62 | 63 | private static Connection getConnection() { 64 | Connection con = null; 65 | try { 66 | Class.forName("com.mysql.jdbc.Driver"); 67 | //注意,替换成自己本地的 mysql 数据库地址和用户名、密码 68 | con = DriverManager.getConnection("jdbc:mysql://localhost:3306/test?useUnicode=true&characterEncoding=UTF-8", "root", "root123456"); 69 | } catch (Exception e) { 70 | System.out.println("-----------mysql get connection has exception , msg = "+ e.getMessage()); 71 | } 72 | return con; 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/java/com/zhisheng/data/sinks/utils/KafkaUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sinks.utils; 2 | 3 | import com.zhisheng.common.utils.GsonUtil; 4 | import com.zhisheng.data.sinks.model.Student; 5 | import org.apache.kafka.clients.producer.KafkaProducer; 6 | import org.apache.kafka.clients.producer.ProducerRecord; 7 | 8 | import java.util.Properties; 9 | 10 | /** 11 | * Desc: 往kafka中写数据,可以使用这个main函数进行测试 12 | * Created by zhisheng on 2019-02-17 13 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 14 | */ 15 | public class KafkaUtil { 16 | public static final String broker_list = "localhost:9092"; 17 | public static final String topic = "student"; //kafka topic 需要和 flink 程序用同一个 topic 18 | 19 | public static void writeToKafka() throws InterruptedException { 20 | Properties props = new Properties(); 21 | props.put("bootstrap.servers", broker_list); 22 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 23 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); 24 | KafkaProducer producer = new KafkaProducer(props); 25 | 26 | for (int i = 1; i <= 100; i++) { 27 | Student student = new Student(i, "zhisheng" + i, "password" + i, 18 + i); 28 | ProducerRecord record = new ProducerRecord(topic, null, null, GsonUtil.toJson(student)); 29 | producer.send(record); 30 | System.out.println("发送数据: " + GsonUtil.toJson(student)); 31 | } 32 | producer.flush(); 33 | } 34 | 35 | public static void main(String[] args) throws InterruptedException { 36 | writeToKafka(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metrics-group 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=student 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-data-sinks/src/main/resources/student.sql: -------------------------------------------------------------------------------- 1 | DROP TABLE IF EXISTS `student`; 2 | CREATE TABLE `student` ( 3 | `id` int(11) unsigned NOT NULL AUTO_INCREMENT, 4 | `name` varchar(25) COLLATE utf8_bin DEFAULT NULL, 5 | `password` varchar(25) COLLATE utf8_bin DEFAULT NULL, 6 | `age` int(10) DEFAULT NULL, 7 | PRIMARY KEY (`id`) 8 | ) ENGINE=InnoDB AUTO_INCREMENT=5 DEFAULT CHARSET=utf8 COLLATE=utf8_bin; 9 | 10 | 11 | -------------------------------------------------------------------------------- /flink-learning-data-sources/README.md: -------------------------------------------------------------------------------- 1 | ## Flink data source 2 | 3 | [http://www.54tianzhisheng.cn/2018/10/28/flink-sources/](http://www.54tianzhisheng.cn/2018/10/28/flink-sources/) 4 | 5 | [http://www.54tianzhisheng.cn/2018/10/30/flink-create-source/](http://www.54tianzhisheng.cn/2018/10/30/flink-create-source/) 6 | 7 | 8 | 9 | 定时任务捞取 MySQL 数据:可以查看 ScheduleMain 类的实现 10 | 11 | ![](http://zhisheng-blog.oss-cn-hangzhou.aliyuncs.com/img/2019-05-24-124853.jpg) -------------------------------------------------------------------------------- /flink-learning-data-sources/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-data-sources 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | mysql 22 | mysql-connector-java 23 | 5.1.34 24 | 25 | 26 | 27 | 28 | 29 | 30 | org.apache.maven.plugins 31 | maven-shade-plugin 32 | 3.1.0 33 | 34 | false 35 | 36 | 37 | 38 | package 39 | 40 | shade 41 | 42 | 43 | 44 | 45 | 46 | 48 | 49 | com.zhisheng.data.sources.Main 50 | 51 | 53 | reference.conf 54 | 55 | 56 | 57 | 58 | *:*:*:* 59 | 60 | META-INF/*.SF 61 | META-INF/*.DSA 62 | META-INF/*.RSA 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources; 2 | 3 | 4 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 5 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 6 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 7 | import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; 8 | 9 | import java.util.Properties; 10 | 11 | /** 12 | * 利用 flink kafka 自带的 source 读取 kafka 里面的数据 13 | */ 14 | public class Main { 15 | public static void main(String[] args) throws Exception{ 16 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 17 | 18 | Properties props = new Properties(); 19 | props.put("bootstrap.servers", "localhost:9092"); 20 | props.put("zookeeper.connect", "localhost:2181"); 21 | props.put("group.id", "metric-group"); 22 | props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); //key 反序列化 23 | props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); 24 | props.put("auto.offset.reset", "latest"); //value 反序列化 25 | 26 | DataStreamSource dataStreamSource = env.addSource(new FlinkKafkaConsumer011<>( 27 | "metric", //kafka topic 28 | new SimpleStringSchema(), // String 序列化 29 | props)).setParallelism(1); 30 | 31 | dataStreamSource.print(); //把从 kafka 读取到的数据打印在控制台 32 | 33 | env.execute("Flink add data source"); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/Main2.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources; 2 | 3 | import com.zhisheng.data.sources.sources.SourceFromMySQL; 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | 6 | /** 7 | * Desc: 自定义 source,从 mysql 中读取数据 8 | * Created by zhisheng on 2019-02-17 9 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 10 | */ 11 | public class Main2 { 12 | public static void main(String[] args) throws Exception { 13 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 14 | 15 | env.addSource(new SourceFromMySQL()).print(); 16 | 17 | env.execute("Flink add data sourc"); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/model/Rule.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Builder; 5 | import lombok.Data; 6 | import lombok.NoArgsConstructor; 7 | 8 | /** 9 | * Desc: 规则 10 | * Created by zhisheng on 2019-05-24 11 | * blog:http://www.54tianzhisheng.cn/ 12 | * 微信公众号:zhisheng 13 | */ 14 | @Data 15 | @AllArgsConstructor 16 | @NoArgsConstructor 17 | @Builder 18 | public class Rule { 19 | 20 | /** 21 | * rule id 22 | */ 23 | private String id; 24 | 25 | /** 26 | * rule name 27 | */ 28 | private String name; 29 | 30 | /** 31 | * rule type 32 | */ 33 | private String type; 34 | 35 | /** 36 | * monitor measurement 37 | */ 38 | private String measurement; 39 | 40 | /** 41 | * rule expression 42 | */ 43 | private String expression; 44 | 45 | /** 46 | * measurement threshold 47 | */ 48 | private String threshold; 49 | 50 | /** 51 | * alert level 52 | */ 53 | private String level; 54 | 55 | /** 56 | * rule targetType 57 | */ 58 | private String targetType; 59 | 60 | /** 61 | * rule targetId 62 | */ 63 | private String targetId; 64 | 65 | /** 66 | * notice webhook, only DingDing group rebot here 67 | * TODO: more notice ways 68 | */ 69 | private String webhook; 70 | } 71 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/model/Student.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-02-17 10 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 11 | */ 12 | 13 | @Data 14 | @AllArgsConstructor 15 | @NoArgsConstructor 16 | public class Student { 17 | public int id; 18 | public String name; 19 | public String password; 20 | public int age; 21 | } 22 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/sources/SourceFromMySQL.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources.sources; 2 | 3 | import com.zhisheng.data.sources.model.Student; 4 | import com.zhisheng.data.sources.utils.MySQLUtil; 5 | import org.apache.flink.configuration.Configuration; 6 | import org.apache.flink.streaming.api.functions.source.RichSourceFunction; 7 | 8 | import java.sql.Connection; 9 | import java.sql.DriverManager; 10 | import java.sql.PreparedStatement; 11 | import java.sql.ResultSet; 12 | 13 | /** 14 | * Desc: 自定义 source,从 mysql 中读取数据 15 | * Created by zhisheng on 2019-02-17 16 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 17 | */ 18 | public class SourceFromMySQL extends RichSourceFunction { 19 | 20 | PreparedStatement ps; 21 | private Connection connection; 22 | 23 | /** 24 | * open() 方法中建立连接,这样不用每次 invoke 的时候都要建立连接和释放连接。 25 | * 26 | * @param parameters 27 | * @throws Exception 28 | */ 29 | @Override 30 | public void open(Configuration parameters) throws Exception { 31 | super.open(parameters); 32 | connection = MySQLUtil.getConnection("com.mysql.jdbc.Driver", 33 | "jdbc:mysql://localhost:3306/test?useUnicode=true&characterEncoding=UTF-8", 34 | "root", 35 | "root123456"); 36 | String sql = "select * from Student;"; 37 | ps = this.connection.prepareStatement(sql); 38 | } 39 | 40 | /** 41 | * 程序执行完毕就可以进行,关闭连接和释放资源的动作了 42 | * 43 | * @throws Exception 44 | */ 45 | @Override 46 | public void close() throws Exception { 47 | super.close(); 48 | if (connection != null) { //关闭连接和释放资源 49 | connection.close(); 50 | } 51 | if (ps != null) { 52 | ps.close(); 53 | } 54 | } 55 | 56 | /** 57 | * DataStream 调用一次 run() 方法用来获取数据 58 | * 59 | * @param ctx 60 | * @throws Exception 61 | */ 62 | @Override 63 | public void run(SourceContext ctx) throws Exception { 64 | ResultSet resultSet = ps.executeQuery(); 65 | while (resultSet.next()) { 66 | Student student = new Student( 67 | resultSet.getInt("id"), 68 | resultSet.getString("name").trim(), 69 | resultSet.getString("password").trim(), 70 | resultSet.getInt("age")); 71 | ctx.collect(student); 72 | } 73 | } 74 | 75 | @Override 76 | public void cancel() { 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/utils/KafkaUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources.utils; 2 | 3 | import com.zhisheng.common.model.MetricEvent; 4 | import com.zhisheng.common.utils.GsonUtil; 5 | import org.apache.kafka.clients.producer.KafkaProducer; 6 | import org.apache.kafka.clients.producer.ProducerRecord; 7 | 8 | import java.util.HashMap; 9 | import java.util.Map; 10 | import java.util.Properties; 11 | 12 | /** 13 | * Desc: 往kafka中写数据,可以使用这个main函数进行测试 14 | * Created by zhisheng on 2019-02-17 15 | * Blog: http://www.54tianzhisheng.cn/tags/Flink/ 16 | */ 17 | public class KafkaUtil { 18 | public static final String broker_list = "localhost:9092"; 19 | public static final String topic = "metric"; // kafka topic,Flink 程序中需要和这个统一 20 | 21 | public static void writeToKafka() throws InterruptedException { 22 | Properties props = new Properties(); 23 | props.put("bootstrap.servers", broker_list); 24 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); //key 序列化 25 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); //value 序列化 26 | KafkaProducer producer = new KafkaProducer(props); 27 | 28 | MetricEvent metric = new MetricEvent(); 29 | metric.setTimestamp(System.currentTimeMillis()); 30 | metric.setName("mem"); 31 | Map tags = new HashMap<>(); 32 | Map fields = new HashMap<>(); 33 | 34 | tags.put("cluster", "zhisheng"); 35 | tags.put("host_ip", "101.147.022.106"); 36 | 37 | fields.put("used_percent", 90d); 38 | fields.put("max", 27244873d); 39 | fields.put("used", 17244873d); 40 | fields.put("init", 27244873d); 41 | 42 | metric.setTags(tags); 43 | metric.setFields(fields); 44 | 45 | ProducerRecord record = new ProducerRecord(topic, null, null, GsonUtil.toJson(metric)); 46 | producer.send(record); 47 | System.out.println("发送数据: " + GsonUtil.toJson(metric)); 48 | 49 | producer.flush(); 50 | } 51 | 52 | public static void main(String[] args) throws InterruptedException { 53 | while (true) { 54 | Thread.sleep(300); 55 | writeToKafka(); 56 | } 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/java/com/zhisheng/data/sources/utils/MySQLUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.data.sources.utils; 2 | 3 | import java.sql.Connection; 4 | import java.sql.DriverManager; 5 | 6 | /** 7 | * Desc: MySQL 工具类 8 | * Created by zhisheng on 2019-05-24 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | public class MySQLUtil { 13 | 14 | public static Connection getConnection(String driver, String url, String user, String password) { 15 | Connection con = null; 16 | try { 17 | Class.forName(driver); 18 | //注意,这里替换成你自己的mysql 数据库路径和用户名、密码 19 | con = DriverManager.getConnection(url, user, password); 20 | } catch (Exception e) { 21 | System.out.println("-----------mysql get connection has exception , msg = "+ e.getMessage()); 22 | } 23 | return con; 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=metrics-group 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=alert-metrics 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/resources/rule.sql: -------------------------------------------------------------------------------- 1 | # ************************************************************ 2 | # Host: 127.0.0.1 (MySQL 5.7.22) 3 | # Database: zhisheng 4 | # Generation Time: 2019-05-24 09:37:57 +0000 5 | # ************************************************************ 6 | 7 | 8 | /*!40101 SET @OLD_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT */; 9 | /*!40101 SET @OLD_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS */; 10 | /*!40101 SET @OLD_COLLATION_CONNECTION=@@COLLATION_CONNECTION */; 11 | /*!40101 SET NAMES utf8 */; 12 | /*!40014 SET @OLD_FOREIGN_KEY_CHECKS=@@FOREIGN_KEY_CHECKS, FOREIGN_KEY_CHECKS=0 */; 13 | /*!40101 SET @OLD_SQL_MODE=@@SQL_MODE, SQL_MODE='NO_AUTO_VALUE_ON_ZERO' */; 14 | /*!40111 SET @OLD_SQL_NOTES=@@SQL_NOTES, SQL_NOTES=0 */; 15 | 16 | 17 | # Dump of table rule 18 | # ------------------------------------------------------------ 19 | 20 | DROP TABLE IF EXISTS `rule`; 21 | 22 | CREATE TABLE `rule` ( 23 | `id` varchar(255) COLLATE utf8_bin NOT NULL DEFAULT '', 24 | `name` varchar(255) COLLATE utf8_bin DEFAULT NULL, 25 | `type` varchar(50) COLLATE utf8_bin DEFAULT NULL, 26 | `measurement` varchar(255) COLLATE utf8_bin DEFAULT NULL, 27 | `expression` varchar(1023) COLLATE utf8_bin DEFAULT NULL, 28 | `threshold` varchar(10) COLLATE utf8_bin DEFAULT NULL, 29 | `level` varchar(12) COLLATE utf8_bin DEFAULT NULL, 30 | `target_type` varchar(12) COLLATE utf8_bin DEFAULT NULL, 31 | `target_id` varchar(24) COLLATE utf8_bin DEFAULT NULL, 32 | `webhook` varchar(255) COLLATE utf8_bin DEFAULT NULL, 33 | PRIMARY KEY (`id`) 34 | ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; 35 | 36 | LOCK TABLES `rule` WRITE; 37 | /*!40000 ALTER TABLE `rule` DISABLE KEYS */; 38 | 39 | INSERT INTO `rule` (`id`, `name`, `type`, `measurement`, `expression`, `threshold`, `level`, `target_type`, `target_id`, `webhook`) 40 | VALUES 41 | (X'31',X'6370755F616C657274',X'637075',X'7573655F70657263656E74',NULL,X'3830',X'4552524F52',X'6D616368696E65',X'7A68697368656E67',X'68747470733A2F2F6F6170692E64696E6774616C6B2E636F6D2F726F626F742F73656E643F6163636573735F746F6B656E3D64386336636431626434663132616436303235653430313965623631613933373233326630396566363336643636316636333431383836393666343136616362'), 42 | (X'32',X'6D656D5F616C657274',X'6D656D',X'7573655F70657263656E74',NULL,X'3835',X'4552524F52',X'6D616368696E65',X'7A68697368656E67',X'68747470733A2F2F6F6170692E64696E6774616C6B2E636F6D2F726F626F742F73656E643F6163636573735F746F6B656E3D64386336636431626434663132616436303235653430313965623631613933373233326630396566363336643636316636333431383836393666343136616362'); 43 | 44 | /*!40000 ALTER TABLE `rule` ENABLE KEYS */; 45 | UNLOCK TABLES; 46 | 47 | 48 | 49 | /*!40111 SET SQL_NOTES=@OLD_SQL_NOTES */; 50 | /*!40101 SET SQL_MODE=@OLD_SQL_MODE */; 51 | /*!40014 SET FOREIGN_KEY_CHECKS=@OLD_FOREIGN_KEY_CHECKS */; 52 | /*!40101 SET CHARACTER_SET_CLIENT=@OLD_CHARACTER_SET_CLIENT */; 53 | /*!40101 SET CHARACTER_SET_RESULTS=@OLD_CHARACTER_SET_RESULTS */; 54 | /*!40101 SET COLLATION_CONNECTION=@OLD_COLLATION_CONNECTION */; 55 | -------------------------------------------------------------------------------- /flink-learning-data-sources/src/main/resources/student.sql: -------------------------------------------------------------------------------- 1 | DROP TABLE IF EXISTS `student`; 2 | CREATE TABLE `student` ( 3 | `id` int(11) unsigned NOT NULL AUTO_INCREMENT, 4 | `name` varchar(25) COLLATE utf8_bin DEFAULT NULL, 5 | `password` varchar(25) COLLATE utf8_bin DEFAULT NULL, 6 | `age` int(10) DEFAULT NULL, 7 | PRIMARY KEY (`id`) 8 | ) ENGINE=InnoDB AUTO_INCREMENT=5 DEFAULT CHARSET=utf8 COLLATE=utf8_bin; 9 | 10 | 11 | 12 | INSERT INTO `student` VALUES ('1', 'zhisheng01', '123456', '18'), ('2', 'zhisheng02', '123', '17'), ('3', 'zhisheng03', '1234', '18'), ('4', 'zhisheng04', '12345', '16'); 13 | COMMIT; 14 | -------------------------------------------------------------------------------- /flink-learning-examples/README.md: -------------------------------------------------------------------------------- 1 | ### Flink-learning-example 2 | 3 | 该 module 存放一些简单的测试用例。 -------------------------------------------------------------------------------- /flink-learning-examples/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-examples 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | 22 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/streaming/File/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.streaming.File; 2 | 3 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | 6 | /** 7 | * 从文件读取数据 & 数据写入到文件 8 | * blog:http://www.54tianzhisheng.cn/ 9 | * 微信公众号:zhisheng 10 | */ 11 | public class Main { 12 | public static void main(String[] args) throws Exception { 13 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 14 | env.setParallelism(1); 15 | 16 | DataStreamSource data = env.readTextFile("file:///usr/local/blink-1.5.1/README.txt"); 17 | data.print(); 18 | 19 | //两种格式都行,另外还支持写入到 hdfs 20 | // data.writeAsText("file:///usr/local/blink-1.5.1/README1.txt"); 21 | data.writeAsText("/usr/local/blink-1.5.1/README1.txt"); 22 | 23 | env.execute(); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/streaming/iteration/util/IterateExampleData.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.streaming.iteration.util; 2 | 3 | /** 4 | * Data for IterateExampleITCase. 5 | */ 6 | public class IterateExampleData { 7 | public static final String INPUT_PAIRS = "(1,40)\n" + "(29,38)\n" + "(11,15)\n" + "(17,39)\n" + "(24,41)\n" + 8 | "(7,33)\n" + "(20,2)\n" + "(11,5)\n" + "(3,16)\n" + "(23,36)\n" + "(15,23)\n" + "(28,13)\n" + "(1,1)\n" + 9 | "(10,6)\n" + "(21,5)\n" + "(14,36)\n" + "(17,15)\n" + "(7,9)"; 10 | 11 | public static final String RESULTS = "((1,40),3)\n" + "((24,41),2)\n" + "((3,16),5)\n" + "((1,1),10)\n" + 12 | "((17,15),4)\n" + "((29,38),2)\n" + "((7,33),3)\n" + "((23,36),3)\n" + "((10,6),6)\n" + "((7,9),5)\n" + 13 | "((11,15),4)\n" + "((20,2),5)\n" + "((15,23),4)\n" + "((21,5),5)\n" + 14 | "((17,39),3)\n" + "((11,5),6)\n" + "((28,13),4)\n" + "((14,36),3)"; 15 | 16 | private IterateExampleData() { 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/streaming/join/WindowJoinSampleData.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.streaming.join; 2 | 3 | import com.zhisheng.examples.util.ThrottledIterator; 4 | import org.apache.flink.api.common.typeinfo.TypeHint; 5 | import org.apache.flink.api.common.typeinfo.TypeInformation; 6 | import org.apache.flink.api.java.tuple.Tuple2; 7 | import org.apache.flink.streaming.api.datastream.DataStream; 8 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 9 | 10 | import java.io.Serializable; 11 | import java.util.Iterator; 12 | import java.util.Random; 13 | 14 | /** 15 | * Sample data for the {@link WindowJoin} example. 16 | * blog:http://www.54tianzhisheng.cn/ 17 | * 微信公众号:zhisheng 18 | */ 19 | public class WindowJoinSampleData { 20 | 21 | static final String[] NAMES = {"tom", "jerry", "alice", "bob", "john", "grace"}; 22 | static final int GRADE_COUNT = 5; 23 | static final int SALARY_MAX = 10000; 24 | 25 | /** 26 | * Continuously generates (name, grade). 27 | */ 28 | public static class GradeSource implements Iterator>, Serializable { 29 | 30 | private final Random rnd = new Random(hashCode()); 31 | 32 | @Override 33 | public boolean hasNext() { 34 | return true; 35 | } 36 | 37 | @Override 38 | public Tuple2 next() { 39 | return new Tuple2<>(NAMES[rnd.nextInt(NAMES.length)], rnd.nextInt(GRADE_COUNT) + 1); 40 | } 41 | 42 | @Override 43 | public void remove() { 44 | throw new UnsupportedOperationException(); 45 | } 46 | 47 | public static DataStream> getSource(StreamExecutionEnvironment env, long rate) { 48 | return env.fromCollection(new ThrottledIterator<>(new GradeSource(), rate), 49 | TypeInformation.of(new TypeHint>() { 50 | })); 51 | } 52 | } 53 | 54 | /** 55 | * Continuously generates (name, salary). 56 | */ 57 | public static class SalarySource implements Iterator>, Serializable { 58 | 59 | private final Random rnd = new Random(hashCode()); 60 | 61 | @Override 62 | public boolean hasNext() { 63 | return true; 64 | } 65 | 66 | @Override 67 | public Tuple2 next() { 68 | return new Tuple2<>(NAMES[rnd.nextInt(NAMES.length)], rnd.nextInt(SALARY_MAX) + 1); 69 | } 70 | 71 | @Override 72 | public void remove() { 73 | throw new UnsupportedOperationException(); 74 | } 75 | 76 | public static DataStream> getSource(StreamExecutionEnvironment env, long rate) { 77 | return env.fromCollection(new ThrottledIterator<>(new SalarySource(), rate), 78 | TypeInformation.of(new TypeHint>() { 79 | })); 80 | } 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/streaming/socket/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.streaming.socket; 2 | 3 | import org.apache.flink.api.common.functions.FlatMapFunction; 4 | import org.apache.flink.api.java.tuple.Tuple2; 5 | import org.apache.flink.streaming.api.datastream.DataStreamSource; 6 | import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; 7 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 8 | import org.apache.flink.util.Collector; 9 | 10 | /** 11 | * Desc: socket 12 | * Created by zhisheng on 2019-04-26 13 | * blog:http://www.54tianzhisheng.cn/ 14 | * 微信公众号:zhisheng 15 | */ 16 | public class Main { 17 | public static void main(String[] args) throws Exception { 18 | //参数检查 19 | if (args.length != 2) { 20 | System.err.println("USAGE:\nSocketTextStreamWordCount "); 21 | return; 22 | } 23 | 24 | String hostname = args[0]; 25 | Integer port = Integer.parseInt(args[1]); 26 | 27 | 28 | // set up the streaming execution environment 29 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 30 | 31 | //获取数据 32 | DataStreamSource stream = env.socketTextStream(hostname, port); 33 | 34 | //计数 35 | SingleOutputStreamOperator> sum = stream.flatMap(new LineSplitter()) 36 | .keyBy(0) 37 | .sum(1); 38 | 39 | sum.print(); 40 | 41 | env.execute("Java WordCount from SocketTextStream Example"); 42 | } 43 | 44 | public static final class LineSplitter implements FlatMapFunction> { 45 | @Override 46 | public void flatMap(String s, Collector> collector) { 47 | String[] tokens = s.toLowerCase().split("\\W+"); 48 | 49 | for (String token: tokens) { 50 | if (token.length() > 0) { 51 | collector.collect(new Tuple2(token, 1)); 52 | } 53 | } 54 | } 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/util/MySQLUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.util; 2 | 3 | import com.google.common.base.Throwables; 4 | import lombok.extern.slf4j.Slf4j; 5 | 6 | import java.sql.Connection; 7 | import java.sql.DriverManager; 8 | 9 | @Slf4j 10 | public class MySQLUtil { 11 | 12 | public static Connection getConnection(String driver, String url, String user, String password) { 13 | Connection con = null; 14 | try { 15 | Class.forName(driver); 16 | con = DriverManager.getConnection(url, user, password); 17 | } catch (Exception e) { 18 | log.error("-----------mysql get connection has exception , msg = " + Throwables.getStackTraceAsString(e)); 19 | } 20 | return con; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /flink-learning-examples/src/main/java/com/zhisheng/examples/util/ThrottledIterator.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.examples.util; 2 | 3 | import java.io.Serializable; 4 | import java.util.Iterator; 5 | 6 | import static java.util.Objects.requireNonNull; 7 | 8 | /** 9 | * A variant of the collection source (emits a sequence of elements as a stream) 10 | * that supports throttling the emission rate. 11 | */ 12 | public class ThrottledIterator implements Iterator, Serializable { 13 | 14 | private static final long serialVersionUID = 1L; 15 | 16 | @SuppressWarnings("NonSerializableFieldInSerializableClass") 17 | private final Iterator source; 18 | 19 | private final long sleepBatchSize; 20 | private final long sleepBatchTime; 21 | 22 | private long lastBatchCheckTime; 23 | private long num; 24 | 25 | public ThrottledIterator(Iterator source, long elementsPerSecond) { 26 | this.source = requireNonNull(source); 27 | 28 | if (!(source instanceof Serializable)) { 29 | throw new IllegalArgumentException("source must be java.io.Serializable"); 30 | } 31 | 32 | if (elementsPerSecond >= 100) { 33 | // how many elements would we emit per 50ms 34 | this.sleepBatchSize = elementsPerSecond / 20; 35 | this.sleepBatchTime = 50; 36 | } 37 | else if (elementsPerSecond >= 1) { 38 | // how long does element take 39 | this.sleepBatchSize = 1; 40 | this.sleepBatchTime = 1000 / elementsPerSecond; 41 | } 42 | else { 43 | throw new IllegalArgumentException("'elements per second' must be positive and not zero"); 44 | } 45 | } 46 | 47 | @Override 48 | public boolean hasNext() { 49 | return source.hasNext(); 50 | } 51 | 52 | @Override 53 | public T next() { 54 | // delay if necessary 55 | if (lastBatchCheckTime > 0) { 56 | if (++num >= sleepBatchSize) { 57 | num = 0; 58 | 59 | final long now = System.currentTimeMillis(); 60 | final long elapsed = now - lastBatchCheckTime; 61 | if (elapsed < sleepBatchTime) { 62 | try { 63 | Thread.sleep(sleepBatchTime - elapsed); 64 | } catch (InterruptedException e) { 65 | // restore interrupt flag and proceed 66 | Thread.currentThread().interrupt(); 67 | } 68 | } 69 | lastBatchCheckTime = now; 70 | } 71 | } else { 72 | lastBatchCheckTime = System.currentTimeMillis(); 73 | } 74 | 75 | return source.next(); 76 | } 77 | 78 | @Override 79 | public void remove() { 80 | throw new UnsupportedOperationException(); 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /flink-learning-examples/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 | -------------------------------------------------------------------------------- /flink-learning-examples/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 | -------------------------------------------------------------------------------- /flink-learning-monitor/README.md: -------------------------------------------------------------------------------- 1 | ## Flink 监控、告警、存储 -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/README.md: -------------------------------------------------------------------------------- 1 | ## Flink 监控告警 2 | 3 | ### Flink JobManager 4 | 5 | 6 | ### Flink TaskManager 7 | 8 | ### Flink Jobs -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-monitor 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor-alert 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-monitor-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/AtMobiles.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import lombok.Data; 4 | 5 | import java.util.List; 6 | 7 | /** 8 | * @ 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | @Data 13 | public class AtMobiles { 14 | /** 15 | * 被@人的手机号 16 | * 17 | * @return 18 | */ 19 | public List atMobiles; 20 | 21 | /** 22 | * @所有人时:true,否则为:false 23 | */ 24 | public Boolean isAtAll; 25 | } 26 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/BaseMessage.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import java.io.Serializable; 4 | 5 | /** 6 | * 请求消息的抽象类 7 | * blog:http://www.54tianzhisheng.cn/ 8 | * 微信公众号:zhisheng 9 | */ 10 | public abstract class BaseMessage implements Serializable { 11 | 12 | public BaseMessage() { 13 | init(); 14 | } 15 | 16 | /** 17 | * 消息类型 18 | */ 19 | protected MessageType msgtype; 20 | 21 | 22 | public MessageType getMsgtype() { 23 | return msgtype; 24 | } 25 | 26 | /** 27 | * 初始化 MessageType 方法 28 | */ 29 | protected abstract void init(); 30 | } 31 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/Email.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | import java.io.File; 8 | import java.util.Map; 9 | import java.util.Set; 10 | 11 | @Data 12 | @NoArgsConstructor 13 | @AllArgsConstructor 14 | public class Email { 15 | /** 16 | * 收件人 17 | */ 18 | private Set to; 19 | 20 | /** 21 | * 邮件主题 22 | */ 23 | private String subject; 24 | 25 | /** 26 | * 邮件正文 27 | */ 28 | private String content; 29 | 30 | /** 31 | * 正文是否是 HTML 32 | */ 33 | private boolean isHtml; 34 | 35 | /** 36 | * 附件路径 37 | */ 38 | private Map attachments; 39 | 40 | /** 41 | * 是否有附件 42 | */ 43 | private boolean isAttachment; 44 | } 45 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/LinkMessage.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * 链接类型钉钉消息 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | @Data 13 | @AllArgsConstructor 14 | @NoArgsConstructor 15 | public class LinkMessage extends BaseMessage { 16 | 17 | public Link link; 18 | 19 | @Override 20 | protected void init() { 21 | this.msgtype = MessageType.link; 22 | } 23 | 24 | @Data 25 | public static class Link { 26 | /** 27 | * 消息简介 28 | */ 29 | private String text; 30 | 31 | /** 32 | * 消息标题 33 | */ 34 | private String title; 35 | 36 | /** 37 | * 封面图片URL 38 | */ 39 | private String picUrl; 40 | 41 | /** 42 | * 消息跳转URL 43 | */ 44 | private String messageUrl; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/MarkDownMessage.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * markdown 类型钉钉消息 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | @Data 13 | @AllArgsConstructor 14 | @NoArgsConstructor 15 | public class MarkDownMessage extends BaseMessage { 16 | 17 | public MarkDownContent markdown; 18 | 19 | public AtMobiles at; 20 | 21 | @Override 22 | protected void init() { 23 | this.msgtype = MessageType.markdown; 24 | } 25 | 26 | 27 | @Data 28 | public static class MarkDownContent { 29 | /** 30 | * 首屏会话透出的展示内容 31 | */ 32 | private String title; 33 | 34 | /** 35 | * markdown格式的消息 36 | */ 37 | private String text; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/MessageType.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | /** 4 | * 消息类型:文本、链接、MarkDown、跳转卡片、消息卡片五种枚举值 5 | * blog:http://www.54tianzhisheng.cn/ 6 | * 微信公众号:zhisheng 7 | */ 8 | public enum MessageType { 9 | /** 10 | * 文本类型 11 | */ 12 | text, 13 | 14 | /** 15 | * 链接类型 16 | */ 17 | link, 18 | 19 | /** 20 | * MarkDown类型 21 | */ 22 | markdown, 23 | 24 | /** 25 | * 跳转卡片类型 26 | */ 27 | actionCard, 28 | 29 | /** 30 | * 消息卡片类型 31 | */ 32 | feedCard; 33 | } 34 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/TextMessage.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * 文本类型钉钉消息 9 | * blog:http://www.54tianzhisheng.cn/ 10 | * 微信公众号:zhisheng 11 | */ 12 | @Data 13 | @AllArgsConstructor 14 | @NoArgsConstructor 15 | public class TextMessage extends BaseMessage { 16 | 17 | /** 18 | * 消息内容 19 | */ 20 | public TextContent text; 21 | 22 | /** 23 | * @ 24 | */ 25 | public AtMobiles at; 26 | 27 | 28 | @Override 29 | protected void init() { 30 | this.msgtype = MessageType.text; 31 | } 32 | 33 | 34 | @Data 35 | public static class TextContent { 36 | /** 37 | * 消息内容 38 | */ 39 | private String content; 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/model/WorkNotify.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.model; 2 | 3 | 4 | import lombok.AllArgsConstructor; 5 | import lombok.Data; 6 | import lombok.NoArgsConstructor; 7 | 8 | /** 9 | * 钉钉工作通知 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | @Data 14 | @NoArgsConstructor 15 | public class WorkNotify { 16 | 17 | /** 18 | * 微应用 id 19 | */ 20 | private Integer agent_id; 21 | 22 | /** 23 | * 接收人列表,逗号分隔 24 | */ 25 | private String userid_list; 26 | 27 | /** 28 | * 部门 id 列表,可选 29 | */ 30 | private String dept_id_list; 31 | 32 | /** 33 | * 是否发送给所有 34 | */ 35 | private Boolean to_all_user; 36 | 37 | /** 38 | * 消息体 39 | */ 40 | private Msg msg; 41 | 42 | @Data 43 | @AllArgsConstructor 44 | public static class Msg { 45 | 46 | /** 47 | * 消息类型 48 | */ 49 | private String msgtype; 50 | 51 | private Text text; 52 | 53 | @Data 54 | @AllArgsConstructor 55 | public static class Text { 56 | /** 57 | * 消息内容 58 | */ 59 | private String content; 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/utils/DingDingAccessTokenUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.utils; 2 | 3 | 4 | import com.google.common.base.Throwables; 5 | import com.google.common.cache.Cache; 6 | import com.google.common.cache.CacheBuilder; 7 | import com.jayway.jsonpath.JsonPath; 8 | import com.zhisheng.common.utils.HttpUtil; 9 | import lombok.extern.slf4j.Slf4j; 10 | 11 | import java.util.concurrent.TimeUnit; 12 | 13 | 14 | /** 15 | * 获取access_token 16 | * https://open-doc.dingtalk.com/microapp/serverapi2/eev437 17 | * blog:http://www.54tianzhisheng.cn/ 18 | * 微信公众号:zhisheng 19 | */ 20 | @Slf4j 21 | public class DingDingAccessTokenUtil { 22 | 23 | public static final String DING_DING_ACCESS_TOKEN = "access_token"; 24 | public static final String DING_DING_GET_TOKEN_URL = "https://oapi.dingtalk.com/gettoken"; 25 | 26 | 27 | public static Cache expireCache = CacheBuilder.newBuilder() 28 | .expireAfterAccess(2, TimeUnit.HOURS) 29 | .build(); 30 | 31 | /** 32 | * 根据微应用 key 和 secret 获取 access_token 33 | * 34 | * @param appKey 35 | * @param appSecret 36 | * @return 37 | */ 38 | public static String getAccessToken(String appKey, String appSecret) { 39 | String accessToken = ""; 40 | String dingDingAccessTocken = expireCache.getIfPresent(DING_DING_ACCESS_TOKEN); 41 | if (dingDingAccessTocken == null || "".equals(dingDingAccessTocken)) { 42 | String result = ""; 43 | try { 44 | result = HttpUtil.doGet(DING_DING_GET_TOKEN_URL + "?appkey=" + appKey + "&appsecret=" + appSecret); 45 | accessToken = JsonPath.read(result, "$.access_token"); 46 | if (accessToken != null && !"".equals(accessToken)) { 47 | expireCache.put(DING_DING_ACCESS_TOKEN, String.valueOf(accessToken)); 48 | log.info("get ding ding access token = {}", String.valueOf(accessToken)); 49 | } 50 | } catch (Exception e) { 51 | log.error("--------------httpclient do get request exception , msg = {}, result = {}", 52 | Throwables.getStackTraceAsString(e), result); 53 | } 54 | } else { 55 | accessToken = dingDingAccessTocken; 56 | } 57 | return accessToken; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/utils/DingDingWorkspaceNoticeUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.utils; 2 | 3 | 4 | /** 5 | * 钉钉工作通知工具类 6 | * https://open-doc.dingtalk.com/microapp/serverapi2/pgoxpy 7 | */ 8 | public class DingDingWorkspaceNoticeUtil { 9 | public static final String workNotifyUrl = "https://oapi.dingtalk.com/topapi/message/corpconversation/asyncsend_v2?access_token="; 10 | 11 | 12 | } 13 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/utils/EmailNoticeUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.utils; 2 | 3 | 4 | /** 5 | * 邮件通知工具类 6 | */ 7 | public class EmailNoticeUtil { 8 | } 9 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/utils/PhoneNoticeUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.utils; 2 | 3 | 4 | /** 5 | * 电话通知工具类 6 | */ 7 | public class PhoneNoticeUtil { 8 | } 9 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-alert/src/main/java/com/zhisheng/alert/utils/SMSNoticeUtil.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.alert.utils; 2 | 3 | 4 | /** 5 | * 短信通知工具类 6 | */ 7 | public class SMSNoticeUtil { 8 | } 9 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-collector/README.md: -------------------------------------------------------------------------------- 1 | ## Flink 监控数据采集 2 | 3 | 因为数据采集是非常重要的,如果在底层数据采集的过程中就断了,那么后面的存储和告警链路将全部失效,所以为了保证后面链路的可用性, 4 | 数据采集的话就不能也是一个 Flink Job,否则因为 Flink 挂了的话,那么就会导致数据采集的 Job 失效,导致整个监控告警链路失效。 5 | 6 | 另外,也不符合 Flink 的特点,Flink 其实更在于计算,有数据源(source)和下发处(sink),这里我们采集数据的话就自己写一个项目利用 7 | Flink 自己暴露的 Rest API 去采集相关数据(JobManager、TaskManager、Job 等),将采集好的数据组织好成一个个 Metrics,然后发送到 Kafka。 -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-collector/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-monitor 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor-collector 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-monitor-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-common/README.md: -------------------------------------------------------------------------------- 1 | ## Flink 监控 common -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-common/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-monitor 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor-common 13 | 14 | 15 | 16 | 17 | com.zhisheng.flink 18 | flink-learning-common 19 | 1.0-SNAPSHOT 20 | 21 | 22 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-common/src/main/java/com/zhisheng/common/model/Job.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.model; 2 | 3 | import lombok.Data; 4 | import lombok.NoArgsConstructor; 5 | 6 | @Data 7 | @NoArgsConstructor 8 | public class Job { 9 | 10 | /** 11 | * job id 12 | */ 13 | private String jid; 14 | 15 | /** 16 | * job name 17 | */ 18 | private String name; 19 | 20 | /** 21 | * job status 22 | */ 23 | private JobStatus state; 24 | 25 | /** 26 | * job start time 27 | */ 28 | private Long startTime; 29 | 30 | /** 31 | * job end time 32 | */ 33 | private Long endTime; 34 | 35 | /** 36 | * job duration time 37 | */ 38 | private Long duration; 39 | 40 | /** 41 | * job last modify time 42 | */ 43 | private Long lastModification; 44 | 45 | /** 46 | * job tasks 47 | */ 48 | private Task tasks; 49 | } 50 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-common/src/main/java/com/zhisheng/common/model/JobStatus.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.model; 2 | 3 | 4 | public enum JobStatus { 5 | /** 6 | * Job is newly created, no task has started to run. 7 | */ 8 | CREATED, 9 | 10 | /** 11 | * Some tasks are scheduled or running, some may be pending, some may be finished. 12 | */ 13 | RUNNING, 14 | 15 | /** 16 | * The job has failed and is currently waiting for the cleanup to complete 17 | */ 18 | FAILING, 19 | 20 | /** 21 | * The job has failed with a non-recoverable task failure 22 | */ 23 | FAILED, 24 | 25 | /** 26 | * Job is being cancelled 27 | */ 28 | CANCELLING, 29 | 30 | /** 31 | * Job has been cancelled 32 | */ 33 | CANCELED, 34 | 35 | /** 36 | * All of the job's tasks have successfully finished 37 | */ 38 | FINISHED, 39 | 40 | /** 41 | * The job is currently undergoing a reset and total restart 42 | */ 43 | RESTARTING, 44 | 45 | /** 46 | * The job has been suspended and is currently waiting for the cleanup to complete 47 | */ 48 | SUSPENDING, 49 | 50 | /** 51 | * The job has been suspended which means that it has been stopped but not been removed from a 52 | * potential HA job store. 53 | */ 54 | SUSPENDED, 55 | 56 | /** 57 | * The job is currently reconciling and waits for task execution report to recover state. 58 | */ 59 | RECONCILING; 60 | } 61 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-common/src/main/java/com/zhisheng/common/model/Task.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.common.model; 2 | 3 | import lombok.Data; 4 | import lombok.NoArgsConstructor; 5 | 6 | @Data 7 | @NoArgsConstructor 8 | public class Task { 9 | /** 10 | * task 总个数 11 | */ 12 | private int total; 13 | 14 | /** 15 | * 处于 created 状态的 task 个数 16 | */ 17 | private int created; 18 | 19 | /** 20 | * 处于 scheduled 状态的 task 个数 21 | */ 22 | private int scheduled; 23 | 24 | /** 25 | * 处于 deploying 状态的 task 个数 26 | */ 27 | private int deploying; 28 | 29 | /** 30 | * 处于 running 状态的 task 个数 31 | */ 32 | private int running; 33 | 34 | /** 35 | * 处于 finished 状态的 task 个数 36 | */ 37 | private int finished; 38 | 39 | /** 40 | * 处于 canceling 状态的 task 个数 41 | */ 42 | private int canceling; 43 | 44 | /** 45 | * 处于 canceled 状态的 task 个数 46 | */ 47 | private int canceled; 48 | 49 | /** 50 | * 处于 failed 状态的 task 个数 51 | */ 52 | private int failed; 53 | 54 | /** 55 | * 处于 reconciling 状态的 task 个数 56 | */ 57 | private int reconciling; 58 | } 59 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-dashboard/README.md: -------------------------------------------------------------------------------- 1 | ## Flink Monitor Dashboard -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-dashboard/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-monitor 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor-dashboard 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-monitor-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-storage/README.md: -------------------------------------------------------------------------------- 1 | ## Flink 监控数据存储 -------------------------------------------------------------------------------- /flink-learning-monitor/flink-learning-monitor-storage/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning-monitor 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor-storage 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-monitor-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | -------------------------------------------------------------------------------- /flink-learning-monitor/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-monitor 13 | pom 14 | 15 | flink-learning-monitor-collector 16 | flink-learning-monitor-alert 17 | flink-learning-monitor-storage 18 | flink-learning-monitor-common 19 | flink-learning-monitor-dashboard 20 | 21 | 22 | 23 | -------------------------------------------------------------------------------- /flink-learning-sql/README.md: -------------------------------------------------------------------------------- 1 | ### Flink-learning-sql 2 | 3 | Flink Table & SQL API -------------------------------------------------------------------------------- /flink-learning-sql/src/main/java/com/zhisheng/model/WC.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.model; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | 7 | /** 8 | * Desc: 9 | * Created by zhisheng on 2019-06-02 10 | * blog:http://www.54tianzhisheng.cn/ 11 | * 微信公众号:zhisheng 12 | */ 13 | @Data 14 | @AllArgsConstructor 15 | @NoArgsConstructor 16 | public class WC { 17 | /** 18 | * word 19 | */ 20 | public String word; 21 | 22 | /** 23 | * 出现的次数 24 | */ 25 | public long count; 26 | } 27 | -------------------------------------------------------------------------------- /flink-learning-sql/src/main/java/com/zhisheng/sql/StreamSQLExample.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.sql; 2 | 3 | import lombok.AllArgsConstructor; 4 | import lombok.Data; 5 | import lombok.NoArgsConstructor; 6 | import org.apache.flink.streaming.api.datastream.DataStream; 7 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 8 | import org.apache.flink.table.api.Table; 9 | import org.apache.flink.table.api.java.StreamTableEnvironment; 10 | 11 | import java.util.Arrays; 12 | 13 | /** 14 | * Desc: Convert DataStreams to Tables 15 | * Created by zhisheng on 2019-06-02 16 | * blog:http://www.54tianzhisheng.cn/ 17 | * 微信公众号:zhisheng 18 | */ 19 | public class StreamSQLExample { 20 | public static void main(String[] args) throws Exception { 21 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 22 | StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); 23 | 24 | DataStream orderA = env.fromCollection(Arrays.asList( 25 | new Order(1L, "beer", 3), 26 | new Order(1L, "diaper", 4), 27 | new Order(3L, "rubber", 2))); 28 | 29 | DataStream orderB = env.fromCollection(Arrays.asList( 30 | new Order(2L, "pen", 3), 31 | new Order(2L, "rubber", 3), 32 | new Order(4L, "beer", 1))); 33 | 34 | Table tableA = tEnv.fromDataStream(orderA, "user, product, amount"); 35 | 36 | tEnv.registerDataStream("OrderB", orderB, "user, product, amount"); 37 | 38 | Table result = tEnv.sqlQuery("SELECT * FROM " + tableA + " WHERE amount > 2 UNION ALL " + 39 | "SELECT * FROM OrderB WHERE amount < 2"); 40 | 41 | tEnv.toAppendStream(result, Order.class).print(); 42 | 43 | env.execute(); 44 | } 45 | 46 | 47 | @Data 48 | @AllArgsConstructor 49 | @NoArgsConstructor 50 | public static class Order { 51 | 52 | public Long user; 53 | 54 | public String product; 55 | 56 | public int amount; 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /flink-learning-sql/src/main/java/com/zhisheng/table/ExplainingTable.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.table; 2 | 3 | import org.apache.flink.api.java.tuple.Tuple2; 4 | import org.apache.flink.streaming.api.datastream.DataStream; 5 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 6 | import org.apache.flink.table.api.Table; 7 | import org.apache.flink.table.api.java.StreamTableEnvironment; 8 | 9 | /** 10 | * Desc: explain table 11 | * Created by zhisheng on 2019-06-13 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class ExplainingTable { 16 | public static void main(String[] args) { 17 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 18 | StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); 19 | 20 | DataStream> stream1 = env.fromElements(new Tuple2<>(1, "hello")); 21 | DataStream> stream2 = env.fromElements(new Tuple2<>(1, "hello")); 22 | 23 | Table table1 = tEnv.fromDataStream(stream1, "count, word"); 24 | Table table2 = tEnv.fromDataStream(stream2, "count, word"); 25 | Table table = table1 26 | .where("LIKE(word, 'F%')") 27 | .unionAll(table2); 28 | 29 | String explanation = tEnv.explain(table); 30 | System.out.println(explanation); 31 | } 32 | } 33 | 34 | 35 | /* 36 | 37 | 运行结果是: 38 | 39 | == Abstract Syntax Tree == 40 | LogicalUnion(all=[true]) 41 | LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')]) 42 | LogicalTableScan(table=[[_DataStreamTable_0]]) 43 | LogicalTableScan(table=[[_DataStreamTable_1]]) 44 | 45 | == Optimized Logical Plan == 46 | DataStreamUnion(all=[true], union all=[count, word]) 47 | DataStreamCalc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')]) 48 | DataStreamScan(table=[[_DataStreamTable_0]]) 49 | DataStreamScan(table=[[_DataStreamTable_1]]) 50 | 51 | == Physical Execution Plan == 52 | Stage 1 : Data Source 53 | content : collect elements with CollectionInputFormat 54 | 55 | Stage 2 : Data Source 56 | content : collect elements with CollectionInputFormat 57 | 58 | Stage 3 : Operator 59 | content : from: (count, word) 60 | ship_strategy : REBALANCE 61 | 62 | Stage 4 : Operator 63 | content : where: (LIKE(word, _UTF-16LE'F%')), select: (count, word) 64 | ship_strategy : FORWARD 65 | 66 | Stage 5 : Operator 67 | content : from: (count, word) 68 | ship_strategy : REBALANCE 69 | 70 | */ 71 | -------------------------------------------------------------------------------- /flink-learning-sql/src/main/java/com/zhisheng/table/WordCountTable.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.table; 2 | 3 | import com.zhisheng.model.WC; 4 | import org.apache.flink.api.java.DataSet; 5 | import org.apache.flink.api.java.ExecutionEnvironment; 6 | import org.apache.flink.table.api.Table; 7 | import org.apache.flink.table.api.java.BatchTableEnvironment; 8 | 9 | /** 10 | * Desc: Convert DataSets to Tables(Use Table API) 11 | * Created by zhisheng on 2019-06-02 12 | * blog:http://www.54tianzhisheng.cn/ 13 | * 微信公众号:zhisheng 14 | */ 15 | public class WordCountTable { 16 | 17 | public static void main(String[] args) throws Exception { 18 | ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment(); 19 | BatchTableEnvironment tEnv = BatchTableEnvironment.create(env); 20 | 21 | DataSet input = env.fromElements( 22 | new WC("Hello", 1), 23 | new WC("zhisheng", 1), 24 | new WC("Hello", 1)); 25 | 26 | Table table = tEnv.fromDataSet(input); 27 | 28 | Table filtered = table 29 | .groupBy("word") 30 | .select("word, count.sum as count") 31 | .filter("count = 2"); 32 | 33 | DataSet result = tEnv.toDataSet(filtered, WC.class); 34 | 35 | result.print(); 36 | } 37 | 38 | 39 | } 40 | -------------------------------------------------------------------------------- /flink-learning-sql/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=5 6 | stream.checkpoint.interval=1000 7 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-sql/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-state/README.md: -------------------------------------------------------------------------------- 1 | 模版项目,不做任何代码编写,方便创建新的 module 时复制 -------------------------------------------------------------------------------- /flink-learning-state/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-state 13 | 14 | 15 | -------------------------------------------------------------------------------- /flink-learning-template/README.md: -------------------------------------------------------------------------------- 1 | 模版项目,不做任何代码编写,方便创建新的 module 时复制 -------------------------------------------------------------------------------- /flink-learning-template/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-template 13 | 14 | 15 | 16 | 17 | org.apache.maven.plugins 18 | maven-shade-plugin 19 | 3.1.0 20 | 21 | false 22 | 23 | 24 | 25 | package 26 | 27 | shade 28 | 29 | 30 | 31 | 32 | 33 | 35 | com.zhisheng.template.Main 36 | 37 | 39 | reference.conf 40 | 41 | 42 | 43 | 44 | *:*:*:* 45 | 46 | META-INF/*.SF 47 | META-INF/*.DSA 48 | META-INF/*.RSA 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | -------------------------------------------------------------------------------- /flink-learning-template/src/main/java/com/zhisheng/template/Main.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.template; 2 | 3 | 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 5 | 6 | public class Main { 7 | public static void main(String[] args) throws Exception { 8 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 9 | 10 | env.execute("flink learning project template"); 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /flink-learning-template/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | kafka.brokers=localhost:9092 2 | kafka.group.id=zhisheng 3 | kafka.zookeeper.connect=localhost:2181 4 | metrics.topic=zhisheng 5 | stream.parallelism=4 6 | stream.sink.parallelism=4 7 | stream.default.parallelism=4 8 | stream.checkpoint.interval=1000 9 | stream.checkpoint.enable=false -------------------------------------------------------------------------------- /flink-learning-template/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /flink-learning-window/README.md: -------------------------------------------------------------------------------- 1 | ### Flink-learning-window 2 | 3 | Flink Window 机制学习 4 | 5 | #### Time Window 6 | 7 | #### Count Window 8 | 9 | 10 | #### Session Window -------------------------------------------------------------------------------- /flink-learning-window/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | flink-learning 7 | com.zhisheng.flink 8 | 1.0-SNAPSHOT 9 | 10 | 4.0.0 11 | 12 | flink-learning-window 13 | 14 | 15 | 16 | com.zhisheng.flink 17 | flink-learning-common 18 | 1.0-SNAPSHOT 19 | 20 | 21 | 22 | -------------------------------------------------------------------------------- /flink-learning-window/src/main/java/com/zhisheng/constant/WindowConstant.java: -------------------------------------------------------------------------------- 1 | package com.zhisheng.constant; 2 | 3 | /** 4 | * Desc: Flink Window 案例用到的常量 5 | * Created by zhisheng on 2019-05-14 6 | * blog:http://www.54tianzhisheng.cn/ 7 | * 微信公众号:zhisheng 8 | */ 9 | public class WindowConstant { 10 | public static final String HOST_NAME = "hostName"; 11 | public static final String PORT = "port"; 12 | } 13 | -------------------------------------------------------------------------------- /flink-learning-window/src/main/resources/application.properties: -------------------------------------------------------------------------------- 1 | hostName: localhost 2 | port: 9000 -------------------------------------------------------------------------------- /flink-learning-window/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /pics/Flink-code.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/pics/Flink-code.png -------------------------------------------------------------------------------- /pics/Flink-learning.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/InterestingLab/flink-learning/aef361562a703382f5822a671260d9a001d69126/pics/Flink-learning.png --------------------------------------------------------------------------------