summaryrefslogtreecommitdiff
path: root/src/main/java/com/zdjizhi/topology/LogFlowWriteTopology.java
blob: 0ed5052f392e9103857a1ad60a100c0a056ff948 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
package com.zdjizhi.topology;

import cn.hutool.core.convert.Convert;
import cn.hutool.core.util.ObjectUtil;
import cn.hutool.log.Log;
import cn.hutool.log.LogFactory;
import com.zdjizhi.common.*;
import com.zdjizhi.enums.DnsType;
import com.zdjizhi.etl.*;
import com.zdjizhi.utils.arangodb.ArangoDBSink;
import com.zdjizhi.utils.ck.ClickhouseSink;
import com.zdjizhi.utils.kafka.KafkaConsumer;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.time.Time;

import java.time.Duration;
import java.util.Map;
import java.util.Objects;

import static com.zdjizhi.common.FlowWriteConfig.*;

public class LogFlowWriteTopology {
    private static final Log logger = LogFactory.get();

    public static void main(String[] args) {
        try {
            final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

            //两个输出之间的最大时间 (单位milliseconds)
            env.setBufferTimeout(FlowWriteConfig.BUFFER_TIMEOUT);

            //1 connection,2 dns
            if (FlowWriteConfig.LOG_NEED_COMPLETE == 1) {
                //connection
                DataStream<Map<String, Object>> connSource = env.addSource(KafkaConsumer.myDeserializationConsumer(SOURCE_KAFKA_TOPIC_CONNECTION))
                        .filter(Objects::nonNull)
                        .setParallelism(SOURCE_PARALLELISM)
                        .name(SOURCE_KAFKA_TOPIC_CONNECTION);

                //sketch
                DataStream<Map<String, Object>> sketchSource = env.addSource(KafkaConsumer.myDeserializationConsumer(SOURCE_KAFKA_TOPIC_SKETCH))
                        .filter(Objects::nonNull)
                        .setParallelism(FlowWriteConfig.SOURCE_PARALLELISM)
                        .name(SOURCE_KAFKA_TOPIC_SKETCH);

                //transform
                DataStream<Map<String, Object>> connTransformStream = connSource
                        .assignTimestampsAndWatermarks(WatermarkStrategy
                                .<Map<String, Object>>forBoundedOutOfOrderness(Duration.ofSeconds(FlowWriteConfig.FLINK_WATERMARK_MAX_ORDERNESS))
                                .withTimestampAssigner((event, timestamp) -> Convert.toLong(event.get("conn_start_time")) * 1000))
                        .keyBy(new IpKeysSelector())
                        .window(TumblingProcessingTimeWindows.of(Time.seconds(FlowWriteConfig.LOG_AGGREGATE_DURATION)))
                        .process(new ConnProcessFunction())
                        .filter(x -> Objects.nonNull(x))
                        .setParallelism(FlowWriteConfig.TRANSFORM_PARALLELISM);

                DataStream<Map<String, Object>> sketchTransformStream = sketchSource.assignTimestampsAndWatermarks(WatermarkStrategy
                        .<Map<String, Object>>forBoundedOutOfOrderness(Duration.ofSeconds(FlowWriteConfig.FLINK_WATERMARK_MAX_ORDERNESS))
                        .withTimestampAssigner((event, timestamp) -> Convert.toLong(event.get("sketch_start_time")) * 1000))
                        .keyBy(new IpKeysSelector())
                        .window(TumblingProcessingTimeWindows.of(Time.seconds(FlowWriteConfig.LOG_AGGREGATE_DURATION)))
                        .process(new SketchProcessFunction())
                        .filter(Objects::nonNull)
                        .setParallelism(FlowWriteConfig.TRANSFORM_PARALLELISM);


                //入Arangodb
                DataStream<Map<String, Object>> ip2ipGraph = connTransformStream.union(sketchTransformStream)
                        .keyBy(new IpKeysSelector())
                        .window(TumblingProcessingTimeWindows.of(Time.seconds(LOG_AGGREGATE_DURATION_GRAPH)))
                        .process(new Ip2IpGraphProcessFunction())
//                        .filter(Objects::nonNull)
                        .setParallelism(TRANSFORM_PARALLELISM);

                //写入CKsink,批量处理
                connSource.windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new CKWindow()).addSink(new ClickhouseSink(SINK_CK_TABLE_CONNECTION)).name("CKSink");
                sketchSource.windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new CKWindow()).addSink(new ClickhouseSink(SINK_CK_TABLE_SKETCH)).name("CKSink");
                sketchTransformStream.windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new CKWindow()).addSink(new ClickhouseSink(SINK_CK_TABLE_RELATION_CONNECTION)).name("CKSink");

                //写入arangodb
                ip2ipGraph.windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new ArangodbIPWindow()).addSink(new ArangoDBSink(R_VISIT_IP2IP)).name(R_VISIT_IP2IP);

            } else if (FlowWriteConfig.LOG_NEED_COMPLETE == 2) {

                DataStream<Map<String, Object>> dnsSource = env.addSource(KafkaConsumer.myDeserializationConsumer(SOURCE_KAFKA_TOPIC_DNS))
                        .filter(Objects::nonNull)
                        .map(new DnsMapFunction())
                        .setParallelism(FlowWriteConfig.SOURCE_PARALLELISM)
                        .name(FlowWriteConfig.SOURCE_KAFKA_TOPIC_DNS);

                DataStream<Map<String, Object>> dnsTransform = dnsSource.assignTimestampsAndWatermarks(WatermarkStrategy
                        .<Map<String, Object>>forBoundedOutOfOrderness(Duration.ofSeconds(FLINK_WATERMARK_MAX_ORDERNESS))
                        .withTimestampAssigner((event, timestamp) -> Convert.toLong(event.get("capture_time")) * 1000))
                        .flatMap(new DnsSplitFlatMapFunction())
                        .keyBy(new DnsGraphKeysSelector())
                        .window(TumblingProcessingTimeWindows.of(Time.seconds(LOG_AGGREGATE_DURATION)))
                        .process(new DnsRelationProcessFunction())
                        .filter(Objects::nonNull)
                        .setParallelism(TRANSFORM_PARALLELISM);

                //dns 原始日志 ck入库
                dnsSource.filter(Objects::nonNull)
                        .setParallelism(FlowWriteConfig.SINK_PARALLELISM)
                        .windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new CKWindow())
                        .addSink(new ClickhouseSink(SINK_CK_TABLE_DNS))
                        .setParallelism(FlowWriteConfig.SINK_PARALLELISM)
                        .name("CKSink");

                //dns 拆分后relation日志 ck入库
                dnsTransform.windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new CKWindow())
                        .addSink(new ClickhouseSink(SINK_CK_TABLE_RELATION_DNS))
                        .setParallelism(SINK_PARALLELISM)
                        .name("CKSink");

                //arango 入库,按record_type分组入不同的表
                DataStream<Map<String, Object>> dnsGraph = dnsTransform.keyBy(new DnsGraphKeysSelector())
                        .window(TumblingProcessingTimeWindows.of(Time.seconds(LOG_AGGREGATE_DURATION_GRAPH)))
                        .process(new DnsGraphProcessFunction())
                        .setParallelism(SINK_PARALLELISM)
                        .filter(Objects::nonNull);

                for (DnsType dnsEnum : DnsType.values()) {
                    dnsGraph.filter(x -> ObjectUtil.equal(dnsEnum.getType(), x.get("record_type")))
                            .windowAll(TumblingProcessingTimeWindows.of(Time.seconds(SINK_BATCH_TIME_OUT))).apply(new ArangodbDnsWindow())
                            .addSink(new ArangoDBSink(dnsEnum.getSink()))
                            .setParallelism(SINK_PARALLELISM)
                            .name("ArangodbSink");
                }

            }

            env.execute(args[0]);
        } catch (Exception e) {
            logger.error("This Flink task start ERROR! Exception information is : {}", e);
        }

    }

}