重构版本
This commit is contained in:
15
pom.xml
15
pom.xml
@@ -5,8 +5,8 @@
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>com.galaxy.tsg</groupId>
|
||||
<artifactId>flink-top-task</artifactId>
|
||||
<version>22-02-22</version>
|
||||
<artifactId>topn-metrics-job</artifactId>
|
||||
<version>23-05-12</version>
|
||||
<repositories>
|
||||
|
||||
<repository>
|
||||
@@ -133,8 +133,13 @@
|
||||
<dependency>
|
||||
<groupId>com.alibaba</groupId>
|
||||
<artifactId>fastjson</artifactId>
|
||||
<version>1.2.70</version>
|
||||
<version>2.0.26</version>
|
||||
</dependency>
|
||||
<!-- <dependency>
|
||||
<groupId>com.alibaba</groupId>
|
||||
<artifactId>fastjson</artifactId>
|
||||
<version>1.2.70</version>
|
||||
</dependency>-->
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-csv</artifactId>
|
||||
@@ -210,14 +215,14 @@
|
||||
<artifactId>maven-shade-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>flink-top-task</id>
|
||||
<id>topn-metrics-job</id>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>shade</goal>
|
||||
</goals>
|
||||
|
||||
<configuration>
|
||||
<finalName>flink-top-task-23-02-22</finalName>
|
||||
<finalName>topn-metrics-job-23-05-12</finalName>
|
||||
<filters>
|
||||
<filter>
|
||||
<!-- Do not copy the signatures in the META-INF folder.
|
||||
|
||||
@@ -1,18 +1,19 @@
|
||||
package com.galaxy.tsg;
|
||||
|
||||
import com.alibaba.fastjson.JSON;
|
||||
import com.galaxy.tsg.function.*;
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import com.galaxy.tsg.pojo.ResultEntity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import com.alibaba.fastjson2.JSON;
|
||||
import com.galaxy.tsg.function.metricsAggregationReduce;
|
||||
import com.galaxy.tsg.function.metricsCalculate;
|
||||
import com.galaxy.tsg.function.topnHotItems;
|
||||
import com.galaxy.tsg.pojo.resultEntity;
|
||||
import com.galaxy.tsg.pojo.sessionEntity;
|
||||
import com.galaxy.tsg.pojo.transformEntity;
|
||||
import com.zdjizhi.utils.StringUtil;
|
||||
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
|
||||
import org.apache.flink.api.common.functions.FilterFunction;
|
||||
import org.apache.flink.api.common.functions.MapFunction;
|
||||
import org.apache.flink.api.java.functions.KeySelector;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.api.java.tuple.Tuple5;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
@@ -22,664 +23,198 @@ import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import static com.galaxy.tsg.config.commonConfig.*;
|
||||
import static com.galaxy.tsg.util.KafkaUtils.*;
|
||||
import static com.galaxy.tsg.util.kafkaUtils.getKafkaConsumer;
|
||||
import static com.galaxy.tsg.util.kafkaUtils.getKafkaSink;
|
||||
|
||||
public class Toptask {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(Toptask.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
//1.创建执行环境
|
||||
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
//指定使用事件时间
|
||||
//env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
|
||||
|
||||
|
||||
DataStream<String> sourceForSession = env.addSource(getKafkaConsumer("SESSION-RECORD-COMPLETED")).setParallelism(KAFKA_CONSUMER_PARALLELISM);
|
||||
WatermarkStrategy<Entity> strategyForSession = WatermarkStrategy
|
||||
.<Entity>forBoundedOutOfOrderness(Duration.ofSeconds(WATERMARK_TIME))
|
||||
.withTimestampAssigner((Entity, timestamp) -> Entity.getCommon_recv_time() * 1000);
|
||||
|
||||
List<String> topics = new LinkedList<>();
|
||||
topics.add("SECURITY-EVENT-COMPLETED");
|
||||
topics.add("PROXY-EVENT-COMPLETED");
|
||||
DataStream<String> sourceForUrl = env.addSource(getKafkaConsumerLists(topics)).setParallelism(KAFKA_CONSUMER_TOPURL_PARALLELISM);
|
||||
WatermarkStrategy<UrlEntity> strategyForSecurity = WatermarkStrategy
|
||||
.<UrlEntity>forBoundedOutOfOrderness(Duration.ofSeconds(WATERMARK_TIME))
|
||||
.withTimestampAssigner((UrlEntity, timestamp) -> UrlEntity.getCommon_recv_time() * 1000);
|
||||
DataStream<String> sourceForSession = env.addSource(getKafkaConsumer(KAFKA_CONSUMER_TOPIC)).setParallelism(KAFKA_CONSUMER_PARALLELISM);
|
||||
WatermarkStrategy<transformEntity> strategyForSession = WatermarkStrategy
|
||||
.<transformEntity>forBoundedOutOfOrderness(Duration.ofSeconds(WATERMARK_TIME))
|
||||
.withTimestampAssigner((transformEntity, timestamp) -> transformEntity.getTimestamp() * 1000);
|
||||
|
||||
|
||||
SingleOutputStreamOperator<Entity> inputForSession = sourceForSession.map(new MapFunction<String, Entity>() {
|
||||
SingleOutputStreamOperator<transformEntity> inputForSession = sourceForSession.map(new MapFunction<String, transformEntity>() {
|
||||
@Override
|
||||
public Entity map(String message) {
|
||||
Entity entity = new Entity();
|
||||
public transformEntity map(String message) {
|
||||
transformEntity transformEntity = new transformEntity();
|
||||
try {
|
||||
entity = JSON.parseObject(message, Entity.class);
|
||||
sessionEntity sessionEntity = JSON.parseObject(message, com.galaxy.tsg.pojo.sessionEntity.class);
|
||||
transformEntity.setServer_ip(sessionEntity.getCommon_server_ip());
|
||||
transformEntity.setClient_ip(sessionEntity.getCommon_client_ip());
|
||||
|
||||
transformEntity.setSubscriber_id(sessionEntity.getCommon_subscriber_id());
|
||||
transformEntity.setFqdn(sessionEntity.getCommon_server_fqdn());
|
||||
transformEntity.setExternal_ip(sessionEntity.getCommon_external_ip());
|
||||
transformEntity.setInternal_ip(sessionEntity.getCommon_internal_ip());
|
||||
transformEntity.setDomain(sessionEntity.getHttp_domain());
|
||||
transformEntity.setDevice_group(sessionEntity.getCommon_device_group());
|
||||
transformEntity.setDevice_id(sessionEntity.getCommon_device_id());
|
||||
transformEntity.setData_center(sessionEntity.getCommon_data_center());
|
||||
transformEntity.setVsys_id(sessionEntity.getCommon_vsys_id());
|
||||
transformEntity.setTimestamp(sessionEntity.getCommon_recv_time());
|
||||
transformEntity.setSessions(sessionEntity.getCommon_sessions());
|
||||
|
||||
transformEntity.setL4_protocol(sessionEntity.getCommon_l4_protocol());
|
||||
|
||||
|
||||
if ((8L & sessionEntity.getCommon_flags()) == 8L) {
|
||||
|
||||
transformEntity.setOut_bytes(sessionEntity.getCommon_c2s_byte_num());
|
||||
transformEntity.setOut_pkts(sessionEntity.getCommon_c2s_pkt_num());
|
||||
transformEntity.setIn_bytes(sessionEntity.getCommon_s2c_byte_num());
|
||||
transformEntity.setIn_pkts(sessionEntity.getCommon_s2c_pkt_num());
|
||||
|
||||
} else {
|
||||
transformEntity.setOut_bytes(sessionEntity.getCommon_s2c_byte_num());
|
||||
transformEntity.setOut_pkts(sessionEntity.getCommon_s2c_pkt_num());
|
||||
transformEntity.setIn_bytes(sessionEntity.getCommon_c2s_byte_num());
|
||||
transformEntity.setIn_pkts(sessionEntity.getCommon_c2s_pkt_num());
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
} catch (Exception e) {
|
||||
LOG.error("Entity Parsing ERROR");
|
||||
entity.setIfError(1);
|
||||
transformEntity.setIfError(1);
|
||||
}
|
||||
return entity;
|
||||
return transformEntity;
|
||||
}
|
||||
}).filter(new FilterFunction<Entity>() {
|
||||
}).filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(Entity entity) throws Exception {
|
||||
public boolean filter(transformEntity entity) throws Exception {
|
||||
|
||||
return entity.ifError != 1;
|
||||
}
|
||||
});
|
||||
|
||||
SingleOutputStreamOperator<UrlEntity> inputForUrl = sourceForUrl.map(new MapFunction<String, UrlEntity>() {
|
||||
|
||||
//clientip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> clientipdStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public UrlEntity map(String message) {
|
||||
UrlEntity entity = new UrlEntity();
|
||||
try {
|
||||
entity = JSON.parseObject(message, UrlEntity.class);
|
||||
|
||||
} catch (Exception e) {
|
||||
LOG.error("Entity Parsing ERROR");
|
||||
entity.setIfError(1);
|
||||
}
|
||||
return entity;
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getL4_protocol()) || "IPv4_TCP".equals(value.getL4_protocol());
|
||||
}
|
||||
}).filter(new FilterFunction<UrlEntity>() {
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStream = clientipdStream.keyBy(new groupBySelector("client_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "client_ip")).setParallelism(TASK_PARALLELISM).name("client_ip");;
|
||||
DataStream<String> Stream = windowedStream.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
Stream.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
|
||||
//serverip聚合TOP
|
||||
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> serveripdStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(UrlEntity entity) throws Exception {
|
||||
|
||||
return entity.ifError != 1;
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getL4_protocol()) || "IPv4_TCP".equals(value.getL4_protocol());
|
||||
}
|
||||
});
|
||||
|
||||
switch (TMP_TEST_TYPE) {
|
||||
case 1:
|
||||
|
||||
//clientip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> clientipdStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStream = clientipdStream.keyBy(new groupBySelector("common_client_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "common_client_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> windoweddStream = windowedStream.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
windoweddStream.addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
|
||||
//serverip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> serveripdStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForServerIp = serveripdStream.keyBy(new groupBySelector("common_server_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "common_server_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> windoweddStreamForServerIp = windowedStreamForServerIp.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
windoweddStreamForServerIp.addSink(getKafkaSink("TOP-SERVER-IP")).setParallelism(3);
|
||||
|
||||
|
||||
//common_internal_ip聚合TOP
|
||||
SingleOutputStreamOperator<Entity> internalStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_internal_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForInternal = internalStream.keyBy(new groupBySelector("common_internal_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "common_internal_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForInternal = windowedStreamForInternal.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
WindoweddStreamForInternal.addSink(getKafkaSink("TOP-INTERNAL-HOST")).setParallelism(3);
|
||||
|
||||
//common_external_ip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> externalStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_external_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForExternal = externalStream.keyBy(new groupBySelector("common_external_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "common_external_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForExternal = windowedStreamForExternal.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
WindoweddStreamForExternal.addSink(getKafkaSink("TOP-EXTERNAL-HOST")).setParallelism(3);
|
||||
|
||||
//http_domain聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> domainStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_domain());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForDomain = domainStream.keyBy(new groupBySelector("http_domain"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "http_domain")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForDomain = windowedStreamForDomain.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
WindoweddStreamForDomain.addSink(getKafkaSink("TOP-WEBSITE-DOMAIN")).setParallelism(3);
|
||||
|
||||
SingleOutputStreamOperator<Entity> userStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_subscriber_id());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
//common_subscriber_id聚合TOP
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForUser = userStream.keyBy(new groupBySelector("common_subscriber_id"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "common_subscriber_id")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForUser = windowedStreamForUser.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
WindoweddStreamForUser.addSink(getKafkaSink("TOP-USER")).setParallelism(3);
|
||||
|
||||
SingleOutputStreamOperator<Entity> appNameStream = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_app_label());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
//common_app_label聚合求全量
|
||||
appNameStream.keyBy(new groupBySelector("common_app_label"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculateForApp()).addSink(getKafkaSink("TRAFFIC-APP-STAT")).setParallelism(TASK_PARALLELISM);
|
||||
|
||||
|
||||
SingleOutputStreamOperator<UrlEntity> UrlStream = inputForUrl.filter(new FilterFunction<UrlEntity>() {
|
||||
@Override
|
||||
public boolean filter(UrlEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_url());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSecurity);
|
||||
|
||||
//url聚合session求top
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForUrl = UrlStream.keyBy(new twoKeySelector())
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new UrlAggregationReduce(), new metricsCalculateForUrl(URL_TOP_LIMIT)).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForUrl = windowedStreamForUrl.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItemsForUrl(URL_TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForUrl.addSink(getKafkaSink("TOP-URLS")).setParallelism(3);
|
||||
|
||||
break;
|
||||
case 2:
|
||||
//datasketch
|
||||
|
||||
|
||||
//clientip聚合TOP
|
||||
SingleOutputStreamOperator<Entity> clientipdStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
|
||||
clientipdStream2.keyBy(new groupBySelector("common_client_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("common_client_ip"), new UserCountWindowResult6())
|
||||
// .setParallelism(TASK_PARALLELISM)
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
//serverip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> serveripdStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
serveripdStream2.keyBy(new groupBySelector("common_server_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("common_server_ip"), new UserCountWindowResult6())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-SERVER-IP")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
|
||||
//common_internal_ip聚合TOP
|
||||
SingleOutputStreamOperator<Entity> internalStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_internal_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
|
||||
internalStream2.keyBy(new groupBySelector("common_internal_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("common_internal_ip"), new UserCountWindowResult6())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-INTERNAL-HOST")).setParallelism(3);
|
||||
//
|
||||
//
|
||||
//
|
||||
//common_external_ip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> externalStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_external_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
externalStream2.keyBy(new groupBySelector("common_external_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("common_external_ip"), new UserCountWindowResult6())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-EXTERNAL-HOST")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
//http_domain聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> domainStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_domain());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
|
||||
domainStream2.keyBy(new groupBySelector("http_domain"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("http_domain"), new UserCountWindowResult6())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-WEBSITE-DOMAIN")).setParallelism(3);
|
||||
//
|
||||
//
|
||||
//
|
||||
//common_subscriber_id聚合TOP
|
||||
SingleOutputStreamOperator<Entity> userStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_subscriber_id());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
userStream2.keyBy(new groupBySelector("common_subscriber_id"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForMetricsAggregate2("common_subscriber_id"), new UserCountWindowResult6())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-USER")).setParallelism(3);
|
||||
//
|
||||
|
||||
|
||||
|
||||
|
||||
//common_app_label聚合求全量
|
||||
SingleOutputStreamOperator<Entity> appNameStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_app_label());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
|
||||
appNameStream2.keyBy(new groupBySelector("common_app_label"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculateForApp())
|
||||
.addSink(getKafkaSink("TRAFFIC-APP-STAT"))
|
||||
.setParallelism(TASK_PARALLELISM);
|
||||
|
||||
|
||||
|
||||
//url聚合session求top
|
||||
SingleOutputStreamOperator<UrlEntity> UrlStream2 = inputForUrl.filter(new FilterFunction<UrlEntity>() {
|
||||
@Override
|
||||
public boolean filter(UrlEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_url());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSecurity);
|
||||
|
||||
|
||||
|
||||
UrlStream2.keyBy(new twoKeySelector())
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new DatasketchForUrlAggregate2(), new UserCountWindowResult7())
|
||||
// .print();
|
||||
.addSink(getKafkaSink("TOP-URLS")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
break;
|
||||
|
||||
case 3:
|
||||
|
||||
SingleOutputStreamOperator<Entity> clientipdStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStream3 = clientipdStream3.keyBy(new groupBySelector("common_client_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "common_client_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> windoweddStream3 = windowedStream3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
windoweddStream3.addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(1);
|
||||
|
||||
|
||||
|
||||
SingleOutputStreamOperator<Entity> serveripdStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForServerIp3 = serveripdStream3.keyBy(new groupBySelector("common_server_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "common_server_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> windoweddStreamForServerIp3 = windowedStreamForServerIp3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
windoweddStreamForServerIp3.addSink(getKafkaSink("TOP-SERVER-IP")).setParallelism(1);
|
||||
|
||||
|
||||
//common_internal_ip聚合TOP
|
||||
SingleOutputStreamOperator<Entity> internalStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_internal_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForInternal3 = internalStream3.keyBy(new groupBySelector("common_internal_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "common_internal_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForInternal3 = windowedStreamForInternal3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForInternal3.addSink(getKafkaSink("TOP-INTERNAL-HOST")).setParallelism(1);
|
||||
|
||||
//common_external_ip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> externalStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_external_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForExternal3= externalStream3.keyBy(new groupBySelector("common_external_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "common_external_ip")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForExternal3 = windowedStreamForExternal3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForExternal3.addSink(getKafkaSink("TOP-EXTERNAL-HOST")).setParallelism(1);
|
||||
|
||||
//http_domain聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<Entity> domainStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_domain());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForDomain3 = domainStream3.keyBy(new groupBySelector("http_domain"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "http_domain")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForDomain3 = windowedStreamForDomain3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForDomain3.addSink(getKafkaSink("TOP-WEBSITE-DOMAIN")).setParallelism(1);
|
||||
|
||||
SingleOutputStreamOperator<Entity> userStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_subscriber_id());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
//common_subscriber_id聚合TOP
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForUser3 = userStream3.keyBy(new groupBySelector("common_subscriber_id"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.aggregate(new metricsAggregation(TOP_LIMIT), new metricsCalculateTest(TOP_LIMIT, "common_subscriber_id")).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForUser3 = windowedStreamForUser3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItems(TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForUser3.addSink(getKafkaSink("TOP-USER")).setParallelism(1);
|
||||
|
||||
SingleOutputStreamOperator<Entity> appNameStream3 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
@Override
|
||||
public boolean filter(Entity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getCommon_app_label());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
|
||||
//common_app_label聚合求全量
|
||||
appNameStream3.keyBy(new groupBySelector("common_app_label"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculateForApp()).addSink(getKafkaSink("TRAFFIC-APP-STAT")).setParallelism(TASK_PARALLELISM);
|
||||
|
||||
|
||||
SingleOutputStreamOperator<UrlEntity> UrlStream3 = inputForUrl.filter(new FilterFunction<UrlEntity>() {
|
||||
@Override
|
||||
public boolean filter(UrlEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getHttp_url());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSecurity);
|
||||
|
||||
//url聚合session求top
|
||||
SingleOutputStreamOperator<ResultEntity> windowedStreamForUrl3 = UrlStream3.keyBy(new twoKeySelector())
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new UrlAggregationReduce(), new metricsCalculateForUrl(URL_TOP_LIMIT)).setParallelism(TASK_PARALLELISM);
|
||||
DataStream<String> WindoweddStreamForUrl3 = windowedStreamForUrl3.keyBy(new oneKeySelector())
|
||||
.process(new TopNHotItemsForUrl(URL_TOP_LIMIT)).setParallelism(1);
|
||||
WindoweddStreamForUrl3.addSink(getKafkaSink("TOP-URLS")).setParallelism(1);
|
||||
|
||||
break;
|
||||
|
||||
|
||||
|
||||
// clientipdStream2.windowAll(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .aggregate(new DatasketchForMetricsAggregate("clientIpSession"), new UserCountWindowResult5())
|
||||
//// .print();
|
||||
// .addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
|
||||
// clientipdStream2.windowAll(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .aggregate(new DatasketchForMetricsAggregate("clientIpPkt"), new UserCountWindowResult5())
|
||||
//// .print();
|
||||
// .addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
//
|
||||
//
|
||||
// clientipdStream2.windowAll(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .aggregate(new DatasketchForMetricsAggregate("clientIpByte"), new UserCountWindowResult5())
|
||||
//// .print();
|
||||
// .addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
//
|
||||
// //clientip聚合TOP
|
||||
//
|
||||
// SingleOutputStreamOperator<Entity> clientipdStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStream2 = clientipdStream2.keyBy(new groupBySelector("common_client_ip"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "common_client_ip")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> windoweddStream2 = windowedStream2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// windoweddStream2.addSink(getKafkaSink("TOP-CLIENT-IP")).setParallelism(3);
|
||||
//
|
||||
// //serverip聚合TOP
|
||||
//
|
||||
// SingleOutputStreamOperator<Entity> serveripdStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return "IPv6_TCP".equals(value.getCommon_l4_protocol()) || "IPv4_TCP".equals(value.getCommon_l4_protocol());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForServerIp2 = serveripdStream2.keyBy(new groupBySelector("common_server_ip"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "common_server_ip")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> windoweddStreamForServerIp2 = windowedStreamForServerIp2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// windoweddStreamForServerIp2.addSink(getKafkaSink("TOP-SERVER-IP")).setParallelism(3);
|
||||
//
|
||||
//
|
||||
// //common_internal_ip聚合TOP
|
||||
// SingleOutputStreamOperator<Entity> internalStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getCommon_internal_ip());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForInternal2 = internalStream2.keyBy(new groupBySelector("common_internal_ip"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "common_internal_ip")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> WindoweddStreamForInternal2 = windowedStreamForInternal2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// WindoweddStreamForInternal2.addSink(getKafkaSink("TOP-INTERNAL-HOST")).setParallelism(3);
|
||||
//
|
||||
// //common_external_ip聚合TOP
|
||||
//
|
||||
// SingleOutputStreamOperator<Entity> externalStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getCommon_external_ip());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForExternal2 = externalStream2.keyBy(new groupBySelector("common_external_ip"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "common_external_ip")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> WindoweddStreamForExternal2 = windowedStreamForExternal2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// WindoweddStreamForExternal2.addSink(getKafkaSink("TOP-EXTERNAL-HOST")).setParallelism(3);
|
||||
//
|
||||
// //http_domain聚合TOP
|
||||
//
|
||||
// SingleOutputStreamOperator<Entity> domainStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getHttp_domain());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForDomain2 = domainStream2.keyBy(new groupBySelector("http_domain"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "http_domain")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> WindoweddStreamForDomain2 = windowedStreamForDomain2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// WindoweddStreamForDomain2.addSink(getKafkaSink("TOP-WEBSITE-DOMAIN")).setParallelism(3);
|
||||
//
|
||||
// SingleOutputStreamOperator<Entity> userStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getCommon_subscriber_id());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
// //common_subscriber_id聚合TOP
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForUser2 = userStream2.keyBy(new groupBySelector("common_subscriber_id"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new DatasketchMetricsCalculate(TOP_LIMIT, "common_subscriber_id")).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> WindoweddStreamForUser2 = windowedStreamForUser2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItems(TOP_LIMIT)).setParallelism(3);
|
||||
// WindoweddStreamForUser2.addSink(getKafkaSink("TOP-USER")).setParallelism(3);
|
||||
//
|
||||
//
|
||||
// //common_app_label聚合求全量
|
||||
// SingleOutputStreamOperator<Entity> appNameStream2 = inputForSession.filter(new FilterFunction<Entity>() {
|
||||
// @Override
|
||||
// public boolean filter(Entity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getCommon_app_label());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSession);
|
||||
//
|
||||
//
|
||||
//
|
||||
// appNameStream2.keyBy(new groupBySelector("common_app_label"))
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new metricsAggregationReduce(), new metricsCalculateForApp()).addSink(getKafkaSink("TRAFFIC-APP-STAT")).setParallelism(TASK_PARALLELISM);
|
||||
//
|
||||
//
|
||||
//
|
||||
// //url聚合session求top
|
||||
// SingleOutputStreamOperator<UrlEntity> UrlStream2 = inputForUrl.filter(new FilterFunction<UrlEntity>() {
|
||||
// @Override
|
||||
// public boolean filter(UrlEntity value) throws Exception {
|
||||
// return StringUtil.isNotEmpty(value.getHttp_url());
|
||||
// }
|
||||
// }).assignTimestampsAndWatermarks(strategyForSecurity);
|
||||
//
|
||||
//
|
||||
// SingleOutputStreamOperator<ResultEntity> windowedStreamForUrl2 = UrlStream2.keyBy(new twoKeySelector())
|
||||
// .window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
// .reduce(new UrlAggregationReduce(), new DatasketchUrlCalculate(URL_TOP_LIMIT)).setParallelism(TASK_PARALLELISM);
|
||||
// DataStream<String> WindoweddStreamForUrl2 = windowedStreamForUrl2.keyBy(new oneKeySelector())
|
||||
// .process(new TopNHotItemsForUrl(URL_TOP_LIMIT)).setParallelism(1);
|
||||
// WindoweddStreamForUrl2.addSink(getKafkaSink("TOP-URLS")).setParallelism(3);
|
||||
//
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
// break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
env.execute("TOP-task");
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForServerIp = serveripdStream.keyBy(new groupBySelector("server_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "server_ip")).setParallelism(TASK_PARALLELISM).name("server_ip");;
|
||||
DataStream<String> StreamForServerIp = windowedStreamForServerIp.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForServerIp.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
|
||||
//common_internal_ip聚合TOP
|
||||
SingleOutputStreamOperator<transformEntity> internalStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getInternal_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForInternal = internalStream.keyBy(new groupBySelector("internal_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "internal_ip")).setParallelism(TASK_PARALLELISM).name("internal_ip");;
|
||||
DataStream<String> StreamForInternal = windowedStreamForInternal.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForInternal.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
//common_external_ip聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> externalStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getExternal_ip());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForExternal = externalStream.keyBy(new groupBySelector("external_ip"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "external_ip")).setParallelism(TASK_PARALLELISM).name("external_ip");;
|
||||
DataStream<String> StreamForExternal = windowedStreamForExternal.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForExternal.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
//http_domain聚合TOP
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> domainStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getDomain());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForDomain = domainStream.keyBy(new groupBySelector("server_domain"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "server_domain")).setParallelism(TASK_PARALLELISM).name("server_domain");;
|
||||
DataStream<String> StreamForDomain = windowedStreamForDomain.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForDomain.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> userStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getSubscriber_id());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
//common_subscriber_id聚合TOP
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForUser = userStream.keyBy(new groupBySelector("subscriber_id"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "subscriber_id")).setParallelism(TASK_PARALLELISM).name("subscriber_id");;
|
||||
DataStream<String> StreamForUser = windowedStreamForUser.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForUser.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
SingleOutputStreamOperator<transformEntity> fqdnStream = inputForSession.filter(new FilterFunction<transformEntity>() {
|
||||
@Override
|
||||
public boolean filter(transformEntity value) throws Exception {
|
||||
return StringUtil.isNotEmpty(value.getFqdn());
|
||||
}
|
||||
}).assignTimestampsAndWatermarks(strategyForSession);
|
||||
|
||||
SingleOutputStreamOperator<resultEntity> windowedStreamForFqdn = fqdnStream.keyBy(new groupBySelector("server_fqdn"))
|
||||
.window(TumblingEventTimeWindows.of(Time.minutes(WINDOW_TIME_MINUTE)))
|
||||
.reduce(new metricsAggregationReduce(), new metricsCalculate(TOP_LIMIT, "server_fqdn")).setParallelism(TASK_PARALLELISM).name("server_fqdn");
|
||||
DataStream<String> StreamForFqdn = windowedStreamForFqdn.keyBy(new oneKeySelector())
|
||||
.process(new topnHotItems(TOP_LIMIT)).setParallelism(ORDERBY_PARALLELISM);
|
||||
StreamForFqdn.addSink(getKafkaSink(KAFKA_PRODUCER_TOPIC)).setParallelism(SINK_PARALLELISM);
|
||||
|
||||
env.execute(JOB_NAME);
|
||||
|
||||
}
|
||||
|
||||
|
||||
public static class groupBySelector implements KeySelector<Entity, Tuple4<String, Long, String, String>> {
|
||||
public static class groupBySelector implements KeySelector<transformEntity, Tuple5<String, Long, String, String, String>> {
|
||||
|
||||
public String key;
|
||||
|
||||
@@ -688,31 +223,32 @@ public class Toptask {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Tuple4<String, Long, String, String> getKey(Entity entity) throws Exception {
|
||||
public Tuple5<String, Long, String, String, String> getKey(transformEntity transformEntity) throws Exception {
|
||||
|
||||
Tuple4<String, Long, String, String> tuple = null;
|
||||
Tuple5<String, Long, String, String, String> tuple = null;
|
||||
transformEntity.setKey_by(key);
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
tuple = new Tuple4<>(entity.getCommon_client_ip(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "client_ip":
|
||||
tuple = new Tuple5<>(transformEntity.getClient_ip(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
tuple = new Tuple4<>(entity.getCommon_server_ip(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "server_ip":
|
||||
tuple = new Tuple5<>(transformEntity.getServer_ip(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
tuple = new Tuple4<>(entity.getCommon_internal_ip(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "internal_ip":
|
||||
tuple = new Tuple5<>(transformEntity.getInternal_ip(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
tuple = new Tuple4<>(entity.getCommon_external_ip(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "external_ip":
|
||||
tuple = new Tuple5<>(transformEntity.getExternal_ip(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
case "http_domain":
|
||||
tuple = new Tuple4<>(entity.getHttp_domain(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "server_domain":
|
||||
tuple = new Tuple5<>(transformEntity.getDomain(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
tuple = new Tuple4<>(entity.getCommon_subscriber_id(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "subscriber_id":
|
||||
tuple = new Tuple5<>(transformEntity.getSubscriber_id(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
case "common_app_label":
|
||||
tuple = new Tuple4<>(entity.getCommon_app_label(), entity.getCommon_vsys_id(), entity.getCommon_device_group(), entity.getCommon_data_center());
|
||||
case "server_fqdn":
|
||||
tuple = new Tuple5<>(transformEntity.getFqdn(), transformEntity.getVsys_id(), transformEntity.getDevice_group(), transformEntity.getData_center(), transformEntity.getDevice_id());
|
||||
break;
|
||||
|
||||
default:
|
||||
@@ -723,19 +259,13 @@ public class Toptask {
|
||||
}
|
||||
|
||||
|
||||
public static class oneKeySelector implements KeySelector<ResultEntity, Tuple1<String>> {
|
||||
public static class oneKeySelector implements KeySelector<resultEntity, Tuple1<String>> {
|
||||
|
||||
@Override
|
||||
public Tuple1<String> getKey(ResultEntity entity) throws Exception {
|
||||
public Tuple1<String> getKey(resultEntity entity) throws Exception {
|
||||
return new Tuple1<>(entity.getOrder_by());
|
||||
}
|
||||
}
|
||||
|
||||
public static class twoKeySelector implements KeySelector<UrlEntity, Tuple2<String, Long>> {
|
||||
|
||||
@Override
|
||||
public Tuple2<String, Long> getKey(UrlEntity entity) throws Exception {
|
||||
return new Tuple2<>(entity.getHttp_url(), entity.getCommon_vsys_id());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -6,44 +6,47 @@ package com.galaxy.tsg.config;
|
||||
public class commonConfig {
|
||||
|
||||
|
||||
public static final String KAFKA_CONSUMER_BROKER = CommonConfigurations.getStringProperty("kafka.consumer.broker");
|
||||
public static final String KAFKA_CONSUMER_GROUP_ID = CommonConfigurations.getStringProperty("kafka.consumer.group.id");
|
||||
public static final String KAFKA_CONSUMER_TOPIC = CommonConfigurations.getStringProperty("kafka.consumer.topic");
|
||||
public static final int KAFKA_CONSUMER_PARALLELISM = CommonConfigurations.getIntProperty("kafka.consumer.parallelism");
|
||||
public static final String KAFKA_CONSUMER_SESSION_TIMEOUT_MS=CommonConfigurations.getStringProperty("kafka.consumer.session.timeout.ms");
|
||||
public static final String KAFKA_CONSUMER_MAX_POLL_RECORD=CommonConfigurations.getStringProperty("kafka.consumer.max.poll.records");
|
||||
public static final String KAFKA_CONSUMER_MAX_PARTITION_FETCH_BYTES=CommonConfigurations.getStringProperty("kafka.consumer.max.partition.fetch.bytes");
|
||||
public static final int KAFKA_CONSUMER_TOPURL_PARALLELISM=CommonConfigurations.getIntProperty("kafka.consumer.topurl.parallelism");
|
||||
public static final String KAFKA_CONSUMER_BROKER = commonConfigurations.getStringProperty("kafka.consumer.broker");
|
||||
public static final String KAFKA_CONSUMER_GROUP_ID = commonConfigurations.getStringProperty("kafka.consumer.group.id");
|
||||
public static final String KAFKA_CONSUMER_TOPIC = commonConfigurations.getStringProperty("kafka.consumer.topic");
|
||||
public static final int KAFKA_CONSUMER_PARALLELISM = commonConfigurations.getIntProperty("kafka.consumer.parallelism");
|
||||
public static final String KAFKA_CONSUMER_SESSION_TIMEOUT_MS= commonConfigurations.getStringProperty("kafka.consumer.session.timeout.ms");
|
||||
public static final String KAFKA_CONSUMER_MAX_POLL_RECORD= commonConfigurations.getStringProperty("kafka.consumer.max.poll.records");
|
||||
public static final String KAFKA_CONSUMER_MAX_PARTITION_FETCH_BYTES= commonConfigurations.getStringProperty("kafka.consumer.max.partition.fetch.bytes");
|
||||
|
||||
|
||||
|
||||
|
||||
public static final String KAFKA_PRODUCER_RETRIES = CommonConfigurations.getStringProperty("kafka.producer.retries");
|
||||
public static final String KAFKA_PRODUCER_LINGER_MS = CommonConfigurations.getStringProperty("kafka.producer.linger.ms");
|
||||
public static final String KAFKA_PRODUCER_REQUEST_TIMEOUT_MS = CommonConfigurations.getStringProperty("kafka.producer.request.timeout.ms");
|
||||
public static final String KAFKA_PRODUCER_BATCH_SIZE = CommonConfigurations.getStringProperty("kafka.producer.batch.size");
|
||||
public static final String KAFKA_PRODUCER_BUFFER_MEMORY = CommonConfigurations.getStringProperty("kafka.producer.buffer.memory");
|
||||
public static final String KAFKA_PRODUCER_MAX_REQUEST_SIZE = CommonConfigurations.getStringProperty("kafka.producer.max.request.size");
|
||||
public static final String KAFKA_PRODUCER_COMPRESSION_TYPE = CommonConfigurations.getStringProperty("kafka.producer.compression.type");
|
||||
public static final String KAFKA_PRODUCER_TOPIC = commonConfigurations.getStringProperty("kafka.producer.topic");
|
||||
public static final String KAFKA_PRODUCER_RETRIES = commonConfigurations.getStringProperty("kafka.producer.retries");
|
||||
public static final String KAFKA_PRODUCER_LINGER_MS = commonConfigurations.getStringProperty("kafka.producer.linger.ms");
|
||||
public static final String KAFKA_PRODUCER_REQUEST_TIMEOUT_MS = commonConfigurations.getStringProperty("kafka.producer.request.timeout.ms");
|
||||
public static final String KAFKA_PRODUCER_BATCH_SIZE = commonConfigurations.getStringProperty("kafka.producer.batch.size");
|
||||
public static final String KAFKA_PRODUCER_BUFFER_MEMORY = commonConfigurations.getStringProperty("kafka.producer.buffer.memory");
|
||||
public static final String KAFKA_PRODUCER_MAX_REQUEST_SIZE = commonConfigurations.getStringProperty("kafka.producer.max.request.size");
|
||||
public static final String KAFKA_PRODUCER_COMPRESSION_TYPE = commonConfigurations.getStringProperty("kafka.producer.compression.type");
|
||||
public static final String KAFKA_PRODUCER_BROKER = commonConfigurations.getStringProperty("kafka_producer_broker");
|
||||
|
||||
|
||||
|
||||
public static final int TASK_PARALLELISM = CommonConfigurations.getIntProperty("task.parallelism");
|
||||
public static final String JOB_NAME = commonConfigurations.getStringProperty("job.name");
|
||||
public static final int TASK_PARALLELISM = commonConfigurations.getIntProperty("task.parallelism");
|
||||
public static final int ORDERBY_PARALLELISM = commonConfigurations.getIntProperty("orderby.parallelism");
|
||||
public static final int SINK_PARALLELISM = commonConfigurations.getIntProperty("sink.parallelism");
|
||||
|
||||
public static final int WATERMARK_TIME = CommonConfigurations.getIntProperty("watermark.time");
|
||||
public static final int WINDOW_TIME_MINUTE = CommonConfigurations.getIntProperty("window.time.minute");
|
||||
public static final int TOP_LIMIT = CommonConfigurations.getIntProperty("top.limit");
|
||||
public static final int URL_TOP_LIMIT = CommonConfigurations.getIntProperty("url.top.limit");
|
||||
public static final int WATERMARK_TIME = commonConfigurations.getIntProperty("watermark.time");
|
||||
public static final int WINDOW_TIME_MINUTE = commonConfigurations.getIntProperty("window.time.minute");
|
||||
public static final int TOP_LIMIT = commonConfigurations.getIntProperty("top.limit");
|
||||
|
||||
public static final String KAFKA_USER = CommonConfigurations.getStringProperty("kafka.user");
|
||||
public static final String KAFKA_PIN = CommonConfigurations.getStringProperty("kafka.pin");
|
||||
public static final int KAFKA_SECURITY = CommonConfigurations.getIntProperty("kafka.security");
|
||||
public static final String TOOLS_LIBRARY = CommonConfigurations.getStringProperty("tools.library");
|
||||
public static final String KAFKA_CONSUMER_USER = commonConfigurations.getStringProperty("kafka.consumer.user");
|
||||
public static final String KAFKA_CONSUMER_PIN = commonConfigurations.getStringProperty("kafka.consumer.pin");
|
||||
public static final int KAFKA_CONSUMER_SECURITY = commonConfigurations.getIntProperty("kafka.consumer.security");
|
||||
public static final String TOOLS_CONSUMER_LIBRARY = commonConfigurations.getStringProperty("tools.consumer.library");
|
||||
|
||||
public static final String KAFKA_PRODUCER_USER = commonConfigurations.getStringProperty("kafka.producer.user");
|
||||
public static final String KAFKA_PRODUCER_PIN = commonConfigurations.getStringProperty("kafka.producer.pin");
|
||||
public static final int KAFKA_PRODUCER_SECURITY = commonConfigurations.getIntProperty("kafka.producer.security");
|
||||
public static final String TOOLS_PRODUCER_LIBRARY = commonConfigurations.getStringProperty("tools.producer.library");
|
||||
|
||||
public static final String KAFKA_PRODUCER_BROKER = CommonConfigurations.getStringProperty("kafka_producer_broker");
|
||||
|
||||
public static final int TMP_TEST_TYPE = CommonConfigurations.getIntProperty("tmp.test.type");
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -9,7 +9,7 @@ import java.util.Properties;
|
||||
* @author Administrator
|
||||
*/
|
||||
|
||||
public final class CommonConfigurations {
|
||||
public final class commonConfigurations {
|
||||
|
||||
private static Properties propService = new Properties();
|
||||
|
||||
@@ -54,7 +54,7 @@ public final class CommonConfigurations {
|
||||
|
||||
static {
|
||||
try {
|
||||
propService.load(CommonConfigurations.class.getClassLoader().getResourceAsStream("common.properties"));
|
||||
propService.load(commonConfigurations.class.getClassLoader().getResourceAsStream("common.properties"));
|
||||
} catch (Exception e) {
|
||||
propService = null;
|
||||
}
|
||||
@@ -1,9 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/14 10:19
|
||||
*/
|
||||
public class ApplabelAggregate {
|
||||
}
|
||||
@@ -1,280 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/8 19:46
|
||||
* 同指标集合成的值
|
||||
*/
|
||||
public class CollectionValue {
|
||||
|
||||
public long c2s_byte_num;
|
||||
public long c2s_pkt_num;
|
||||
public long s2c_byte_num ;
|
||||
public long s2c_pkt_num ;
|
||||
public long session_num ;
|
||||
public long stat_time ;
|
||||
public long vsys_id ;
|
||||
public String data_center;
|
||||
public String device_group;
|
||||
public String order_by;
|
||||
public String source;
|
||||
|
||||
private String destination ;
|
||||
private String domain;
|
||||
private String subscriber_id;
|
||||
private String app_name;
|
||||
|
||||
|
||||
|
||||
public CollectionValue(long c2s_byte_num, long c2s_pkt_num, long s2c_byte_num, long s2c_pkt_num, long session_num, long stat_time, long vsys_id, String data_center, String device_group, String order_by, String source) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
this.session_num = session_num;
|
||||
this.stat_time = stat_time;
|
||||
this.vsys_id = vsys_id;
|
||||
this.data_center = data_center;
|
||||
this.device_group = device_group;
|
||||
this.order_by = order_by;
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
|
||||
public CollectionValue(Entity entity,String orderby,String key ) {
|
||||
|
||||
|
||||
|
||||
this.c2s_byte_num = entity.getCommon_c2s_byte_num();
|
||||
this.c2s_pkt_num = entity.getCommon_c2s_pkt_num();
|
||||
this.s2c_byte_num = entity.getCommon_s2c_byte_num();
|
||||
this.s2c_pkt_num = entity.getCommon_s2c_pkt_num();
|
||||
this.session_num = entity.getCommon_sessions();
|
||||
this.stat_time = System.currentTimeMillis() / 1000;
|
||||
this.vsys_id = entity.getCommon_vsys_id();
|
||||
this.data_center = entity.getCommon_data_center();
|
||||
this.device_group = entity.getCommon_device_group();
|
||||
this.order_by = orderby;
|
||||
|
||||
this.source = entity.getCommon_client_ip();//分情况????????????
|
||||
|
||||
|
||||
switch(key) {
|
||||
case "common_client_ip":
|
||||
this.source =entity.getCommon_client_ip();
|
||||
break;
|
||||
case "common_server_ip":
|
||||
this.destination =entity.getCommon_server_ip();
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
this.source =entity.getCommon_internal_ip();
|
||||
break;
|
||||
case "common_external_ip":
|
||||
this.destination =entity.getCommon_external_ip();
|
||||
break;
|
||||
case "common_subscriber_id":
|
||||
this.subscriber_id =entity.getCommon_subscriber_id();
|
||||
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
this.app_name =entity.getCommon_app_label();
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
public CollectionValue(Entity entity,String orderby ) {
|
||||
|
||||
|
||||
this.c2s_byte_num = entity.getCommon_c2s_byte_num();
|
||||
this.c2s_pkt_num = entity.getCommon_c2s_pkt_num();
|
||||
this.s2c_byte_num = entity.getCommon_s2c_byte_num();
|
||||
this.s2c_pkt_num = entity.getCommon_s2c_pkt_num();
|
||||
this.session_num = entity.getCommon_sessions();
|
||||
this.stat_time = System.currentTimeMillis() / 1000;
|
||||
this.vsys_id = entity.getCommon_vsys_id();
|
||||
this.data_center = entity.getCommon_data_center();
|
||||
this.device_group = entity.getCommon_device_group();
|
||||
this.order_by = orderby;
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
public CollectionValue getCollectionValue(CollectionValue collectionValue,Entity entity){
|
||||
|
||||
collectionValue.setC2s_byte_num(collectionValue.getC2s_byte_num()+entity.getCommon_c2s_byte_num());
|
||||
|
||||
collectionValue.setC2s_pkt_num(collectionValue.getC2s_pkt_num()+entity.getCommon_c2s_pkt_num());
|
||||
|
||||
collectionValue.setS2c_byte_num(collectionValue.getS2c_byte_num()+entity.getCommon_s2c_byte_num());
|
||||
|
||||
collectionValue.setS2c_pkt_num(collectionValue.getS2c_pkt_num()+entity.getCommon_s2c_pkt_num());
|
||||
|
||||
collectionValue.setSession_num(collectionValue.getSession_num()+entity.getCommon_sessions());
|
||||
|
||||
return collectionValue;
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
public long getC2s_byte_num() {
|
||||
return c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setC2s_byte_num(long c2s_byte_num) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
}
|
||||
|
||||
public long getC2s_pkt_num() {
|
||||
return c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setC2s_pkt_num(long c2s_pkt_num) {
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
}
|
||||
|
||||
public long getS2c_byte_num() {
|
||||
return s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setS2c_byte_num(long s2c_byte_num) {
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
}
|
||||
|
||||
public long getS2c_pkt_num() {
|
||||
return s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setS2c_pkt_num(long s2c_pkt_num) {
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
}
|
||||
|
||||
public long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
public long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public String getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
public void setSource(String source) {
|
||||
this.source = source;
|
||||
|
||||
}
|
||||
|
||||
|
||||
public String getDestination() {
|
||||
return destination;
|
||||
}
|
||||
|
||||
public void setDestination(String destination) {
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public String getApp_name() {
|
||||
return app_name;
|
||||
}
|
||||
|
||||
public void setApp_name(String app_name) {
|
||||
this.app_name = app_name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "CollectionValue{" +
|
||||
"c2s_byte_num=" + c2s_byte_num +
|
||||
", c2s_pkt_num=" + c2s_pkt_num +
|
||||
", s2c_byte_num=" + s2c_byte_num +
|
||||
", s2c_pkt_num=" + s2c_pkt_num +
|
||||
", session_num=" + session_num +
|
||||
", stat_time=" + stat_time +
|
||||
", vsys_id=" + vsys_id +
|
||||
", data_center='" + data_center + '\'' +
|
||||
", device_group='" + device_group + '\'' +
|
||||
", order_by='" + order_by + '\'' +
|
||||
", source='" + source + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -1,147 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.common.functions.AggregateFunction;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/2/14 17:29
|
||||
*
|
||||
*Session_record top10000 21个窗口计算
|
||||
*/
|
||||
public class DatasketchForMetricsAggregate2 implements AggregateFunction<Entity, HashMap<String,DimensionItemsSketch>, HashMap<String,DimensionItemsSketch>> {
|
||||
|
||||
private String key ;
|
||||
|
||||
|
||||
public DatasketchForMetricsAggregate2(String key ){
|
||||
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> createAccumulator() {
|
||||
return new HashMap<String, DimensionItemsSketch>(32768);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> add(Entity entity, HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
// String dimension = cnRecordLog.getCommon_client_ip();//维度
|
||||
// System.out.println(dimension);
|
||||
|
||||
|
||||
if(stringItemsSketchHashMap.isEmpty()) {
|
||||
|
||||
ItemsSketch<String> sessionItemsSketch = new ItemsSketch<>(32768);//新建
|
||||
ItemsSketch<String> pktItemsSketch = new ItemsSketch<>(32768);//新建
|
||||
ItemsSketch<String> byteItemsSketch = new ItemsSketch<>(32768);//新建
|
||||
|
||||
sessionItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_sessions());
|
||||
pktItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_pkt_num()+entity.getCommon_s2c_pkt_num());
|
||||
byteItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_byte_num()+entity.getCommon_s2c_byte_num());
|
||||
|
||||
|
||||
CollectionValue sessionsCollectionValue = new CollectionValue(entity,"sessions",key);
|
||||
// sessionsCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
HashMap<String, CollectionValue> sessionsStringCollectionValueHashMap = new HashMap<>();
|
||||
sessionsStringCollectionValueHashMap.put(Dimension.setDimensionValue(entity,key),sessionsCollectionValue);
|
||||
|
||||
CollectionValue packetsCollectionValue = new CollectionValue(entity,"packets",key);
|
||||
// packetsCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
HashMap<String, CollectionValue> packetsStringCollectionValueHashMap = new HashMap<>();
|
||||
packetsStringCollectionValueHashMap.put(Dimension.setDimensionValue(entity,key),packetsCollectionValue);
|
||||
|
||||
CollectionValue bytesCollectionValue = new CollectionValue(entity,"bytes",key);
|
||||
// bytesCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
HashMap<String, CollectionValue> bytesStringCollectionValueHashMap = new HashMap<>();
|
||||
bytesStringCollectionValueHashMap.put(Dimension.setDimensionValue(entity,key),bytesCollectionValue);
|
||||
|
||||
|
||||
|
||||
DimensionItemsSketch sessionDimensionItemsSketch = new DimensionItemsSketch(Dimension.setDimensionTag(key), sessionItemsSketch,sessionsStringCollectionValueHashMap);
|
||||
DimensionItemsSketch packetsDimensionItemsSketch = new DimensionItemsSketch(Dimension.setDimensionTag(key), pktItemsSketch,packetsStringCollectionValueHashMap);
|
||||
DimensionItemsSketch bytesDimensionItemsSketch = new DimensionItemsSketch(Dimension.setDimensionTag(key), byteItemsSketch,bytesStringCollectionValueHashMap);
|
||||
|
||||
|
||||
stringItemsSketchHashMap.put(key+"Session", sessionDimensionItemsSketch);
|
||||
stringItemsSketchHashMap.put(key+"Packets", packetsDimensionItemsSketch);
|
||||
stringItemsSketchHashMap.put(key+"Bytes", bytesDimensionItemsSketch);
|
||||
|
||||
|
||||
|
||||
|
||||
}else {
|
||||
|
||||
stringItemsSketchHashMap.get(key+"Session").getItemsSketch().update(Dimension.setDimensionValue(entity,key),entity.getCommon_sessions());
|
||||
stringItemsSketchHashMap.get(key+"Packets").getItemsSketch().update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_pkt_num()+entity.getCommon_s2c_pkt_num());
|
||||
stringItemsSketchHashMap.get(key+"Bytes").getItemsSketch().update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_byte_num()+entity.getCommon_s2c_byte_num());
|
||||
|
||||
|
||||
|
||||
CollectionValue sessionsCollectionValue = stringItemsSketchHashMap.get(key+"Session").getStringCollectionValueHashMap().get(Dimension.setDimensionValue(entity,key));//从key获取集合
|
||||
CollectionValue packetsCollectionValue = stringItemsSketchHashMap.get(key+"Packets").getStringCollectionValueHashMap().get(Dimension.setDimensionValue(entity,key));//从key获取集合
|
||||
CollectionValue bytesCollectionValue = stringItemsSketchHashMap.get(key+"Bytes").getStringCollectionValueHashMap().get(Dimension.setDimensionValue(entity,key));//从key获取集合
|
||||
|
||||
|
||||
|
||||
if (sessionsCollectionValue==null){
|
||||
|
||||
sessionsCollectionValue = new CollectionValue(entity,"sessions",key);
|
||||
sessionsCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
|
||||
packetsCollectionValue = new CollectionValue(entity,"packets",key);
|
||||
packetsCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
|
||||
bytesCollectionValue = new CollectionValue(entity,"bytes",key);
|
||||
bytesCollectionValue.setSource(entity.getCommon_client_ip());
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get(key+"Session").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),sessionsCollectionValue);
|
||||
stringItemsSketchHashMap.get(key+"Packets").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),packetsCollectionValue);
|
||||
stringItemsSketchHashMap.get(key+"Bytes").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),bytesCollectionValue);
|
||||
|
||||
|
||||
|
||||
}else {//做加和
|
||||
|
||||
sessionsCollectionValue.getCollectionValue(sessionsCollectionValue,entity);
|
||||
packetsCollectionValue.getCollectionValue(packetsCollectionValue,entity);
|
||||
bytesCollectionValue.getCollectionValue(bytesCollectionValue,entity);
|
||||
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get(key+"Session").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),sessionsCollectionValue);
|
||||
stringItemsSketchHashMap.get(key+"Packets").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),packetsCollectionValue);
|
||||
stringItemsSketchHashMap.get(key+"Bytes").getStringCollectionValueHashMap().put(Dimension.setDimensionValue(entity,key),bytesCollectionValue);
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> getResult(HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> merge(HashMap<String, DimensionItemsSketch> stringDimensionItemsSketchHashMap, HashMap<String, DimensionItemsSketch> acc1) {
|
||||
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -1,107 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.common.functions.AggregateFunction;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/2/14 17:29
|
||||
*
|
||||
*Session_record top10000 21个窗口计算
|
||||
*/
|
||||
public class DatasketchForUrlAggregate implements AggregateFunction<UrlEntity, HashMap<String,DimensionItemsSketch>, HashMap<String,DimensionItemsSketch>> {
|
||||
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> createAccumulator() {
|
||||
return new HashMap<String, DimensionItemsSketch>(1048576);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> add(UrlEntity urlEntity, HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
// String dimension = cnRecordLog.getCommon_client_ip();//维度
|
||||
// System.out.println(dimension);
|
||||
|
||||
if(stringItemsSketchHashMap.isEmpty()) {
|
||||
|
||||
ItemsSketch<String> urlSessionItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
|
||||
urlSessionItemsSketch.update(Dimension.setUrlDimension(urlEntity),urlEntity.getCommon_sessions());
|
||||
|
||||
|
||||
TopUrlEntity topUrlEntity = new TopUrlEntity();
|
||||
topUrlEntity.setSession_num(urlEntity.getCommon_sessions());
|
||||
topUrlEntity.setStat_time(System.currentTimeMillis() / 1000);
|
||||
topUrlEntity.setUrl(urlEntity.getHttp_url());
|
||||
topUrlEntity.setVsys_id(urlEntity.getCommon_vsys_id());
|
||||
HashMap<String,TopUrlEntity> stringTopUrlEntityHashMap = new HashMap<>();
|
||||
stringTopUrlEntityHashMap.put(Dimension.setUrlDimension(urlEntity),topUrlEntity);
|
||||
|
||||
|
||||
DimensionItemsSketch urlSessionDimensionItemsSketch = new DimensionItemsSketch(stringTopUrlEntityHashMap,Dimension.URL,urlSessionItemsSketch);
|
||||
|
||||
stringItemsSketchHashMap.put("urlSession", urlSessionDimensionItemsSketch);
|
||||
|
||||
|
||||
}else {
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getItemsSketch().update(Dimension.setUrlDimension(urlEntity),urlEntity.getCommon_sessions());
|
||||
|
||||
TopUrlEntity urlSession = stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().get(Dimension.setUrlDimension(urlEntity));//从key获取集合
|
||||
|
||||
if (urlSession==null){
|
||||
|
||||
TopUrlEntity topUrlEntity = new TopUrlEntity();
|
||||
topUrlEntity.setSession_num(urlEntity.getCommon_sessions());
|
||||
topUrlEntity.setStat_time(System.currentTimeMillis() / 1000);
|
||||
topUrlEntity.setUrl(urlEntity.getHttp_url());
|
||||
topUrlEntity.setVsys_id(urlEntity.getCommon_vsys_id());
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().put(Dimension.setUrlDimension(urlEntity),urlSession);
|
||||
|
||||
}else {//做加和
|
||||
|
||||
|
||||
urlSession.setSession_num(urlSession.getSession_num()+urlEntity.getCommon_sessions());
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().put(Dimension.setUrlDimension(urlEntity),urlSession);
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> getResult(HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> merge(HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap, HashMap<String, DimensionItemsSketch> acc1) {
|
||||
|
||||
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -1,107 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.common.functions.AggregateFunction;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/2/14 17:29
|
||||
*
|
||||
*Session_record top10000 21个窗口计算
|
||||
*/
|
||||
public class DatasketchForUrlAggregate2 implements AggregateFunction<UrlEntity, HashMap<String,DimensionItemsSketch>, HashMap<String,DimensionItemsSketch>> {
|
||||
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> createAccumulator() {
|
||||
return new HashMap<String, DimensionItemsSketch>(1048576);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> add(UrlEntity urlEntity, HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
// String dimension = cnRecordLog.getCommon_client_ip();//维度
|
||||
// System.out.println(dimension);
|
||||
|
||||
if(stringItemsSketchHashMap.isEmpty()) {
|
||||
|
||||
ItemsSketch<String> urlSessionItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
|
||||
urlSessionItemsSketch.update(Dimension.setUrlDimension(urlEntity),urlEntity.getCommon_sessions());
|
||||
|
||||
|
||||
TopUrlEntity topUrlEntity = new TopUrlEntity();
|
||||
topUrlEntity.setSession_num(urlEntity.getCommon_sessions());
|
||||
topUrlEntity.setStat_time(System.currentTimeMillis() / 1000);
|
||||
topUrlEntity.setUrl(urlEntity.getHttp_url());
|
||||
topUrlEntity.setVsys_id(urlEntity.getCommon_vsys_id());
|
||||
HashMap<String,TopUrlEntity> stringTopUrlEntityHashMap = new HashMap<>();
|
||||
stringTopUrlEntityHashMap.put(Dimension.setUrlDimension(urlEntity),topUrlEntity);
|
||||
|
||||
|
||||
DimensionItemsSketch urlSessionDimensionItemsSketch = new DimensionItemsSketch(stringTopUrlEntityHashMap,Dimension.URL,urlSessionItemsSketch);
|
||||
|
||||
stringItemsSketchHashMap.put("urlSession", urlSessionDimensionItemsSketch);
|
||||
|
||||
|
||||
}else {
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getItemsSketch().update(Dimension.setUrlDimension(urlEntity),urlEntity.getCommon_sessions());
|
||||
|
||||
TopUrlEntity urlSession = stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().get(Dimension.setUrlDimension(urlEntity));//从key获取集合
|
||||
|
||||
if (urlSession==null){
|
||||
|
||||
TopUrlEntity topUrlEntity = new TopUrlEntity();
|
||||
topUrlEntity.setSession_num(urlEntity.getCommon_sessions());
|
||||
topUrlEntity.setStat_time(System.currentTimeMillis() / 1000);
|
||||
topUrlEntity.setUrl(urlEntity.getHttp_url());
|
||||
topUrlEntity.setVsys_id(urlEntity.getCommon_vsys_id());
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().put(Dimension.setUrlDimension(urlEntity),urlSession);
|
||||
|
||||
}else {//做加和
|
||||
|
||||
|
||||
urlSession.setSession_num(urlSession.getSession_num()+urlEntity.getCommon_sessions());
|
||||
|
||||
|
||||
stringItemsSketchHashMap.get("urlSession").getStringTopUrlEntityHashMap().put(Dimension.setUrlDimension(urlEntity),urlSession);
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> getResult(HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap) {
|
||||
|
||||
|
||||
return stringItemsSketchHashMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, DimensionItemsSketch> merge(HashMap<String, DimensionItemsSketch> stringItemsSketchHashMap, HashMap<String, DimensionItemsSketch> acc1) {
|
||||
|
||||
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -1,275 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.*;
|
||||
import org.apache.datasketches.frequencies.ErrorType;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
import static com.galaxy.tsg.config.commonConfig.TOP_LIMIT;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/10 16:09
|
||||
*/
|
||||
public class DatasketchMetricsCalculate extends ProcessWindowFunction<
|
||||
Entity, // 输入类型
|
||||
ResultEntity, // 输出类型
|
||||
Tuple4<String,Long,String,String>, // 键类型
|
||||
TimeWindow> {
|
||||
|
||||
private final int topSize;
|
||||
private final String key;
|
||||
|
||||
|
||||
|
||||
|
||||
public DatasketchMetricsCalculate(int topSize, String key) {
|
||||
this.topSize = topSize;
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Tuple4<String, Long, String, String> S, Context context, Iterable<Entity> iterable, Collector<ResultEntity> collector) throws Exception {
|
||||
|
||||
|
||||
ItemsSketch<String> sessionItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
ItemsSketch<String> pktItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
ItemsSketch<String> byteItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
|
||||
HashMap<String, SessionResultEntity> sessionResultEntityHashMap = new HashMap<>();
|
||||
HashMap<String, PacketResultEntity> packetResultEntityHashMap = new HashMap<>();
|
||||
HashMap<String, ByteResultEntity> byteResultEntityHashMap = new HashMap<>();
|
||||
|
||||
|
||||
|
||||
for (Entity entity:iterable){
|
||||
|
||||
//处理session
|
||||
sessionItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_sessions());
|
||||
sessionResultEntityHashMap.put(Dimension.setDimensionValue(entity,key),enrichessionResult(context.window().getEnd() / 1000, entity));
|
||||
|
||||
//处理pkt
|
||||
pktItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_pkt_num()+entity.getCommon_s2c_pkt_num());
|
||||
packetResultEntityHashMap.put(Dimension.setDimensionValue(entity,key),enrichPacketResult(context.window().getEnd() / 1000, entity));
|
||||
|
||||
//处理byte
|
||||
byteItemsSketch.update(Dimension.setDimensionValue(entity,key),entity.getCommon_c2s_byte_num()+entity.getCommon_s2c_byte_num());
|
||||
byteResultEntityHashMap.put(Dimension.setDimensionValue(entity,key),enrichByteResult(context.window().getEnd() / 1000, entity));
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
ItemsSketch.Row<String>[] sessionFrequentItems = sessionItemsSketch.getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i=0;i<sessionFrequentItems.length;i++){
|
||||
|
||||
SessionResultEntity sessionResultEntity = sessionResultEntityHashMap.get(sessionFrequentItems[i].getItem());
|
||||
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("sessions");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setSessionResultEntity(sessionResultEntity);
|
||||
collector.collect(en);
|
||||
|
||||
|
||||
if (i==topSize)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
ItemsSketch.Row<String>[] pktFrequentItems = pktItemsSketch.getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i=0;i<pktFrequentItems.length;i++){
|
||||
|
||||
PacketResultEntity packetResultEntity = packetResultEntityHashMap.get(pktFrequentItems[i].getItem());
|
||||
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("packets");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setPacketResultEntity(packetResultEntity);
|
||||
collector.collect(en);
|
||||
|
||||
|
||||
if (i==topSize)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
ItemsSketch.Row<String>[] byteFrequentItems = byteItemsSketch.getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i=0;i<byteFrequentItems.length;i++){
|
||||
|
||||
ByteResultEntity byteResultEntity = byteResultEntityHashMap.get(byteFrequentItems[i].getItem());
|
||||
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("bytes");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setByteResultEntity(byteResultEntity);
|
||||
collector.collect(en);
|
||||
|
||||
|
||||
if (i==TOP_LIMIT)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
public ByteResultEntity enrichByteResult(Long time,Entity objectEntity) {
|
||||
ByteResultEntity en = new ByteResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setOrder_by("bytes");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "http_domain":
|
||||
en.setDomain(objectEntity.getHttp_domain());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
}
|
||||
return en;
|
||||
|
||||
}
|
||||
public SessionResultEntity enrichessionResult(Long time,Entity objectEntity){
|
||||
|
||||
SessionResultEntity en =new SessionResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setOrder_by("sessions");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch(key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "http_domain":
|
||||
en.setDomain(objectEntity.getHttp_domain());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
|
||||
|
||||
}
|
||||
return en;
|
||||
}
|
||||
public PacketResultEntity enrichPacketResult(Long time,Entity objectEntity){
|
||||
PacketResultEntity en =new PacketResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setOrder_by("packets");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch(key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
|
||||
}
|
||||
return en;
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,96 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.ResultEntity;
|
||||
import com.galaxy.tsg.pojo.SessionResultEntity;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import org.apache.datasketches.frequencies.ErrorType;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/10 18:27
|
||||
*/
|
||||
public class DatasketchUrlCalculate extends ProcessWindowFunction<
|
||||
UrlEntity, // 输入类型
|
||||
ResultEntity, // 输出类型
|
||||
Tuple2<String,Long>, // 键类型
|
||||
TimeWindow> {
|
||||
|
||||
private final int topSize;
|
||||
|
||||
|
||||
|
||||
public DatasketchUrlCalculate(int topSize) {
|
||||
this.topSize = topSize;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Tuple2<String, Long> stringLongTuple2, Context context, Iterable<UrlEntity> iterable, Collector<ResultEntity> collector) throws Exception {
|
||||
|
||||
|
||||
ItemsSketch<String> sessionItemsSketch = new ItemsSketch<>(1048576);//新建
|
||||
HashMap<String, TopUrlEntity> sessionResultEntityHashMap = new HashMap<>();
|
||||
|
||||
|
||||
|
||||
|
||||
for (UrlEntity urlEntity:iterable){
|
||||
|
||||
|
||||
//处理session
|
||||
sessionItemsSketch.update(Dimension.setUrlDimension(urlEntity),urlEntity.getCommon_sessions());
|
||||
|
||||
TopUrlEntity en = new TopUrlEntity();
|
||||
en.setSession_num(urlEntity.getCommon_sessions());
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setUrl(urlEntity.getHttp_url());
|
||||
en.setVsys_id(urlEntity.getCommon_vsys_id());
|
||||
|
||||
sessionResultEntityHashMap.put(Dimension.setUrlDimension(urlEntity),en);
|
||||
|
||||
}
|
||||
|
||||
|
||||
ItemsSketch.Row<String>[] sessionFrequentItems = sessionItemsSketch.getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i=0;i<sessionFrequentItems.length;i++){
|
||||
|
||||
TopUrlEntity topUrlEntity = sessionResultEntityHashMap.get(sessionFrequentItems[i].getItem());
|
||||
|
||||
ResultEntity resultEntity = new ResultEntity();
|
||||
resultEntity.setOrder_by("sessions");
|
||||
resultEntity.setStat_time(context.window().getEnd() / 1000);
|
||||
resultEntity.setTopUrlEntity(topUrlEntity);
|
||||
collector.collect(resultEntity);
|
||||
|
||||
|
||||
if (i==topSize)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,165 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/2/17 10:14
|
||||
* 各种维度
|
||||
*/
|
||||
public class Dimension {
|
||||
|
||||
public static final String CLIENTIP = "common_client_ip,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String SERVERIP = "common_server_ip,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String INTERNALIP = "common_internal_ip,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String EXTERNALIP = "common_external_ip,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String DOMAIN = "http_domain,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String SUBSCRIBERID = "common_subscriber_id,common_vsys_id,common_device_group,common_data_center";
|
||||
public static final String APPLABEL = "common_app_label,common_vsys_id,common_device_group,common_data_center";
|
||||
|
||||
public static final String URL = "http_url,common_vsys_id";
|
||||
|
||||
|
||||
|
||||
public static String setClientIpDimension(Entity entity){
|
||||
|
||||
String clientIpDimension = entity.getCommon_client_ip()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return clientIpDimension;
|
||||
}
|
||||
|
||||
|
||||
public static String setServerIpDimension(Entity entity){
|
||||
|
||||
String serverIpDimension = entity.getCommon_server_ip()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return serverIpDimension;
|
||||
}
|
||||
|
||||
|
||||
public static String setInternalIpDimension(Entity entity){
|
||||
|
||||
String internalIpDimension = entity.getCommon_internal_ip()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return internalIpDimension;
|
||||
}
|
||||
|
||||
|
||||
public static String setExternalIpDimension(Entity entity){
|
||||
|
||||
String externalIpDimension = entity.getCommon_external_ip()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return externalIpDimension;
|
||||
}
|
||||
|
||||
public static String setDomainDimension(Entity entity){
|
||||
|
||||
String domainDimension = entity.getHttp_domain()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return domainDimension;
|
||||
}
|
||||
|
||||
|
||||
public static String setSubscriberIdDimension(Entity entity){
|
||||
|
||||
String subscriberIdDimension = entity.getCommon_subscriber_id()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return subscriberIdDimension;
|
||||
}
|
||||
|
||||
|
||||
public static String setAppLabelDimension(Entity entity){
|
||||
|
||||
String appLabelDimension = entity.getCommon_app_label()+","+entity.getCommon_vsys_id()+","+entity.getCommon_device_group()+","+entity.getCommon_data_center();
|
||||
return appLabelDimension;
|
||||
}
|
||||
|
||||
public static String setUrlDimension(UrlEntity entity){
|
||||
|
||||
String urlDimension = entity.getHttp_url()+","+entity.getCommon_vsys_id();
|
||||
return urlDimension;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public static String setDimensionValue(Entity entity, String key){
|
||||
|
||||
String dimension = "";
|
||||
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
dimension = setClientIpDimension(entity);
|
||||
break;
|
||||
case "common_server_ip":
|
||||
dimension = setServerIpDimension(entity);
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
dimension = setInternalIpDimension(entity);
|
||||
break;
|
||||
case "common_external_ip":
|
||||
dimension = setExternalIpDimension(entity);
|
||||
break;
|
||||
case "http_domain":
|
||||
dimension = setDomainDimension(entity);
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
dimension = setSubscriberIdDimension(entity);
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
dimension = setAppLabelDimension(entity);
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
return dimension;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public static String setDimensionTag(String key){
|
||||
|
||||
String dimension = "";
|
||||
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
dimension = CLIENTIP;
|
||||
break;
|
||||
case "common_server_ip":
|
||||
dimension = SERVERIP;
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
dimension = INTERNALIP;
|
||||
break;
|
||||
case "common_external_ip":
|
||||
dimension = EXTERNALIP;
|
||||
break;
|
||||
case "http_domain":
|
||||
dimension = DOMAIN;
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
dimension = SUBSCRIBERID;
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
dimension = APPLABEL;
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
return dimension;
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,82 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/2/17 10:03
|
||||
* 维度+datasketches
|
||||
*/
|
||||
public class DimensionItemsSketch {
|
||||
|
||||
private String dimension;//维度
|
||||
private ItemsSketch<String> itemsSketch;//对应的
|
||||
|
||||
private HashMap<String,CollectionValue> stringCollectionValueHashMap;//同指标集合成的值
|
||||
|
||||
private HashMap<String, TopUrlEntity> stringTopUrlEntityHashMap;//同指标集合成的值
|
||||
|
||||
|
||||
|
||||
public DimensionItemsSketch(String dimension, ItemsSketch<String> itemsSketch) {
|
||||
this.dimension = dimension;
|
||||
this.itemsSketch = itemsSketch;
|
||||
}
|
||||
|
||||
|
||||
public DimensionItemsSketch(String dimension, ItemsSketch<String> itemsSketch, HashMap<String, CollectionValue> stringCollectionValueHashMap) {
|
||||
this.dimension = dimension;
|
||||
this.itemsSketch = itemsSketch;
|
||||
this.stringCollectionValueHashMap = stringCollectionValueHashMap;
|
||||
}
|
||||
|
||||
|
||||
public DimensionItemsSketch(HashMap<String, TopUrlEntity> stringTopUrlEntityHashMap, String dimension, ItemsSketch<String> itemsSketch) {
|
||||
this.dimension = dimension;
|
||||
this.itemsSketch = itemsSketch;
|
||||
this.stringTopUrlEntityHashMap = stringTopUrlEntityHashMap;
|
||||
}
|
||||
|
||||
public String getDimension() {
|
||||
return dimension;
|
||||
}
|
||||
|
||||
public void setDimension(String dimension) {
|
||||
this.dimension = dimension;
|
||||
}
|
||||
|
||||
public ItemsSketch<String> getItemsSketch() {
|
||||
return itemsSketch;
|
||||
}
|
||||
|
||||
public void setItemsSketch(ItemsSketch<String> itemsSketch) {
|
||||
this.itemsSketch = itemsSketch;
|
||||
}
|
||||
|
||||
|
||||
public HashMap<String, CollectionValue> getStringCollectionValueHashMap() {
|
||||
return stringCollectionValueHashMap;
|
||||
}
|
||||
|
||||
public void setStringCollectionValueHashMap(HashMap<String, CollectionValue> stringCollectionValueHashMap) {
|
||||
this.stringCollectionValueHashMap = stringCollectionValueHashMap;
|
||||
}
|
||||
|
||||
|
||||
public HashMap<String, TopUrlEntity> getStringTopUrlEntityHashMap() {
|
||||
return stringTopUrlEntityHashMap;
|
||||
}
|
||||
|
||||
public void setStringTopUrlEntityHashMap(HashMap<String, TopUrlEntity> stringTopUrlEntityHashMap) {
|
||||
this.stringTopUrlEntityHashMap = stringTopUrlEntityHashMap;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
@@ -1,139 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.ByteResultEntity;
|
||||
import com.galaxy.tsg.pojo.PacketResultEntity;
|
||||
import com.galaxy.tsg.pojo.ResultEntity;
|
||||
import com.galaxy.tsg.pojo.SessionResultEntity;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class TopNHotItems extends KeyedProcessFunction<Tuple1<String>, ResultEntity, String> {
|
||||
private final int topSize;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<SessionResultEntity> sessionOrderEntity ;
|
||||
private PriorityQueue<PacketResultEntity> packetOrderEntity ;
|
||||
private PriorityQueue<ByteResultEntity> byteOrderEntity ;
|
||||
|
||||
|
||||
public TopNHotItems(int i) {
|
||||
this.topSize = i;
|
||||
|
||||
|
||||
|
||||
}
|
||||
@Override
|
||||
public void open(Configuration parameters) {
|
||||
|
||||
this.sessionOrderEntity=new PriorityQueue<>();
|
||||
this.packetOrderEntity=new PriorityQueue<>();
|
||||
this.byteOrderEntity=new PriorityQueue<>();
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void processElement(ResultEntity objectEntity, Context context, Collector<String> collector) {
|
||||
|
||||
|
||||
|
||||
|
||||
switch(objectEntity.getOrder_by()) {
|
||||
case "sessions":
|
||||
|
||||
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
sessionOrderEntity.add(objectEntity.getSessionResultEntity());
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
SessionResultEntity res=sessionOrderEntity.peek();
|
||||
if (res.getSession_num() <= objectEntity.getSessionResultEntity().getSession_num()) {
|
||||
sessionOrderEntity.poll();
|
||||
sessionOrderEntity.add(objectEntity.getSessionResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
break;
|
||||
case "packets":
|
||||
if (packetOrderEntity.size() < topSize) {
|
||||
packetOrderEntity.add(objectEntity.getPacketResultEntity());
|
||||
} else {
|
||||
if (packetOrderEntity.peek() != null) {
|
||||
PacketResultEntity res=packetOrderEntity.peek();
|
||||
if ((res.getS2c_pkt_num()+res.getC2s_pkt_num()) <= (objectEntity.getPacketResultEntity().getC2s_pkt_num()+objectEntity.getPacketResultEntity().getS2c_pkt_num())) {
|
||||
packetOrderEntity.poll();
|
||||
packetOrderEntity.add(objectEntity.getPacketResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
case "bytes":
|
||||
if (byteOrderEntity.size() < topSize) {
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
} else {
|
||||
if (byteOrderEntity.peek() != null) {
|
||||
ByteResultEntity res=byteOrderEntity.peek();
|
||||
if ((res.getC2s_byte_num()+res.getS2c_byte_num()) <= (objectEntity.getByteResultEntity().getS2c_byte_num()+objectEntity.getByteResultEntity().getC2s_byte_num())) {
|
||||
byteOrderEntity.poll();
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
if (byteOrderEntity.size() < topSize) {
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
} else {
|
||||
if (byteOrderEntity.peek() != null) {
|
||||
ByteResultEntity res=byteOrderEntity.peek();
|
||||
if ((res.getC2s_byte_num()+res.getS2c_byte_num()) <= (objectEntity.getByteResultEntity().getS2c_byte_num()+objectEntity.getByteResultEntity().getC2s_byte_num())) {
|
||||
byteOrderEntity.poll();
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//注册 windowEnd+1 的 EventTime Timer, 当触发时,说明收集好了所有 windowEnd的商品数据
|
||||
context.timerService().registerEventTimeTimer(objectEntity.getStat_time() + 1);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onTimer(long timestamp, OnTimerContext ctx, Collector<String> out) {
|
||||
|
||||
for(SessionResultEntity en : sessionOrderEntity){
|
||||
|
||||
String jsonStr = JSONObject.toJSONString(en);
|
||||
out.collect(jsonStr);
|
||||
|
||||
}
|
||||
|
||||
|
||||
for(PacketResultEntity en : packetOrderEntity){
|
||||
|
||||
String jsonStr = JSONObject.toJSONString(en);
|
||||
out.collect(jsonStr);
|
||||
|
||||
}
|
||||
for(ByteResultEntity en : byteOrderEntity){
|
||||
|
||||
String jsonStr = JSONObject.toJSONString(en);
|
||||
out.collect(jsonStr);
|
||||
|
||||
}
|
||||
|
||||
sessionOrderEntity.clear();
|
||||
packetOrderEntity.clear();
|
||||
byteOrderEntity.clear();
|
||||
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
@@ -1,68 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.ResultEntity;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class TopNHotItemsForUrl extends KeyedProcessFunction<Tuple1<String>, ResultEntity, String> {
|
||||
private final int topSize;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<TopUrlEntity> sessionOrderEntity ;
|
||||
|
||||
|
||||
public TopNHotItemsForUrl(int i) {
|
||||
this.topSize = i;
|
||||
|
||||
|
||||
|
||||
}
|
||||
@Override
|
||||
public void open(Configuration parameters) {
|
||||
|
||||
this.sessionOrderEntity=new PriorityQueue<>();
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void processElement(ResultEntity objectEntity, Context context, Collector<String> collector) {
|
||||
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
sessionOrderEntity.add(objectEntity.getTopUrlEntity());
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
TopUrlEntity res=sessionOrderEntity.peek();
|
||||
if (res.getSession_num() <= objectEntity.getTopUrlEntity().getSession_num()) {
|
||||
sessionOrderEntity.poll();
|
||||
sessionOrderEntity.add(objectEntity.getTopUrlEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
//注册 windowEnd+1 的 EventTime Timer, 当触发时,说明收集好了所有 windowEnd的商品数据
|
||||
context.timerService().registerEventTimeTimer(objectEntity.getStat_time() + 1);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onTimer(long timestamp, OnTimerContext ctx, Collector<String> out) {
|
||||
|
||||
for(TopUrlEntity en : sessionOrderEntity){
|
||||
|
||||
String jsonStr = JSONObject.toJSONString(en);
|
||||
out.collect(jsonStr);
|
||||
|
||||
}
|
||||
|
||||
|
||||
sessionOrderEntity.clear();
|
||||
|
||||
}
|
||||
}
|
||||
@@ -1,13 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import org.apache.flink.api.common.functions.ReduceFunction;
|
||||
|
||||
public class UrlAggregationReduce implements ReduceFunction<UrlEntity> {
|
||||
|
||||
@Override
|
||||
public UrlEntity reduce(UrlEntity value1, UrlEntity value2) throws Exception {
|
||||
value1.setCommon_sessions(value1.getCommon_sessions()+value2.getCommon_sessions());
|
||||
return value1;
|
||||
}
|
||||
}
|
||||
@@ -1,95 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import org.apache.datasketches.frequencies.ErrorType;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.sql.Timestamp;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static com.galaxy.tsg.config.commonConfig.TOP_LIMIT;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/7 15:50
|
||||
*/
|
||||
public class UserCountWindowResult5 extends ProcessAllWindowFunction<HashMap<String, DimensionItemsSketch>, String, TimeWindow> {
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Context context, Iterable<HashMap<String, DimensionItemsSketch>> iterable, Collector<String> collector) throws Exception {
|
||||
// StringBuilder stringBuilder = new StringBuilder();
|
||||
// stringBuilder.append("-------------------------\n");
|
||||
// stringBuilder.append("datasketches方法窗口结束时间:" + new Timestamp(context.window().getEnd()) + "\n");
|
||||
|
||||
|
||||
HashMap<String, DimensionItemsSketch> dataHashMap = iterable.iterator().next();
|
||||
|
||||
|
||||
// System.out.println(dataHashMap.toString());
|
||||
|
||||
Set<Map.Entry<String, DimensionItemsSketch>> entries = dataHashMap.entrySet();
|
||||
|
||||
|
||||
for(Map.Entry<String, DimensionItemsSketch> entry:entries){
|
||||
|
||||
// System.out.println(entry.getKey()+"");
|
||||
// stringBuilder.append(entry.getKey()+"\n");
|
||||
|
||||
ItemsSketch.Row<String>[] items = entry.getValue().getItemsSketch().getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i=0;i<items.length;i++){
|
||||
|
||||
|
||||
// String resultStr = "No." + (i + 1) + " "
|
||||
// + "ip:" + items[i].getItem() + " "
|
||||
// + " Est:" + items[i].getEstimate()
|
||||
// + " UB:" + items[i].getUpperBound()
|
||||
// + " LB:" + items[i].getLowerBound();
|
||||
|
||||
String jsonStr = "";
|
||||
if(!entry.getKey().equals("urlSession")){
|
||||
CollectionValue collectionValue = entry.getValue().getStringCollectionValueHashMap().get(items[i].getItem());
|
||||
collectionValue.setStat_time(context.window().getEnd()/1000);
|
||||
jsonStr = JSONObject.toJSONString(collectionValue);
|
||||
}else {
|
||||
TopUrlEntity topUrlEntity = entry.getValue().getStringTopUrlEntityHashMap().get(items[i].getItem());
|
||||
topUrlEntity.setStat_time(context.window().getEnd()/1000);
|
||||
jsonStr = JSONObject.toJSONString(topUrlEntity);
|
||||
}
|
||||
|
||||
|
||||
collector.collect(jsonStr);
|
||||
|
||||
// String item = items[i].toString();
|
||||
// stringBuilder.append(resultStr);
|
||||
|
||||
// stringBuilder.append("\n");
|
||||
|
||||
|
||||
if (i==TOP_LIMIT)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
// collector.collect(stringBuilder.toString());
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,89 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import org.apache.datasketches.frequencies.ErrorType;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static com.galaxy.tsg.config.commonConfig.TOP_LIMIT;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/7 15:50
|
||||
*/
|
||||
public class UserCountWindowResult6 extends ProcessWindowFunction<HashMap<String, DimensionItemsSketch>, String, Tuple4<String, Long, String, String>,TimeWindow> {
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Tuple4<String, Long, String, String> stringLongStringStringTuple4, Context context, Iterable<HashMap<String, DimensionItemsSketch>> iterable, Collector<String> collector) throws Exception {
|
||||
|
||||
|
||||
HashMap<String, DimensionItemsSketch> dataHashMap = iterable.iterator().next();
|
||||
|
||||
|
||||
// System.out.println(dataHashMap.toString());
|
||||
|
||||
Set<Map.Entry<String, DimensionItemsSketch>> entries = dataHashMap.entrySet();
|
||||
|
||||
|
||||
for (Map.Entry<String, DimensionItemsSketch> entry : entries) {
|
||||
|
||||
// System.out.println(entry.getKey()+"");
|
||||
// stringBuilder.append(entry.getKey()+"\n");
|
||||
|
||||
ItemsSketch.Row<String>[] items = entry.getValue().getItemsSketch().getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i = 0; i < items.length; i++) {
|
||||
|
||||
|
||||
// String resultStr = "No." + (i + 1) + " "
|
||||
// + "ip:" + items[i].getItem() + " "
|
||||
// + " Est:" + items[i].getEstimate()
|
||||
// + " UB:" + items[i].getUpperBound()
|
||||
// + " LB:" + items[i].getLowerBound();
|
||||
|
||||
String jsonStr = "";
|
||||
if (!entry.getKey().equals("urlSession")) {
|
||||
CollectionValue collectionValue = entry.getValue().getStringCollectionValueHashMap().get(items[i].getItem());
|
||||
collectionValue.setStat_time(context.window().getEnd() / 1000);
|
||||
jsonStr = JSONObject.toJSONString(collectionValue);
|
||||
} else {
|
||||
TopUrlEntity topUrlEntity = entry.getValue().getStringTopUrlEntityHashMap().get(items[i].getItem());
|
||||
topUrlEntity.setStat_time(context.window().getEnd() / 1000);
|
||||
jsonStr = JSONObject.toJSONString(topUrlEntity);
|
||||
}
|
||||
|
||||
|
||||
collector.collect(jsonStr);
|
||||
|
||||
// String item = items[i].toString();
|
||||
// stringBuilder.append(resultStr);
|
||||
|
||||
// stringBuilder.append("\n");
|
||||
|
||||
|
||||
if (i == TOP_LIMIT)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -1,87 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import org.apache.datasketches.frequencies.ErrorType;
|
||||
import org.apache.datasketches.frequencies.ItemsSketch;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static com.galaxy.tsg.config.commonConfig.TOP_LIMIT;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2023/3/7 15:50
|
||||
*/
|
||||
public class UserCountWindowResult7 extends ProcessWindowFunction<HashMap<String, DimensionItemsSketch>, String, Tuple2<String, Long>,TimeWindow> {
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Tuple2<String, Long> stringLongTuple2, Context context, Iterable<HashMap<String, DimensionItemsSketch>> iterable, Collector<String> collector) throws Exception {
|
||||
|
||||
|
||||
HashMap<String, DimensionItemsSketch> dataHashMap = iterable.iterator().next();
|
||||
|
||||
|
||||
// System.out.println(dataHashMap.toString());
|
||||
|
||||
Set<Map.Entry<String, DimensionItemsSketch>> entries = dataHashMap.entrySet();
|
||||
|
||||
|
||||
for (Map.Entry<String, DimensionItemsSketch> entry : entries) {
|
||||
|
||||
// System.out.println(entry.getKey()+"");
|
||||
// stringBuilder.append(entry.getKey()+"\n");
|
||||
|
||||
ItemsSketch.Row<String>[] items = entry.getValue().getItemsSketch().getFrequentItems(ErrorType.NO_FALSE_POSITIVES);
|
||||
|
||||
for (int i = 0; i < items.length; i++) {
|
||||
|
||||
|
||||
// String resultStr = "No." + (i + 1) + " "
|
||||
// + "ip:" + items[i].getItem() + " "
|
||||
// + " Est:" + items[i].getEstimate()
|
||||
// + " UB:" + items[i].getUpperBound()
|
||||
// + " LB:" + items[i].getLowerBound();
|
||||
|
||||
String jsonStr = "";
|
||||
if (!entry.getKey().equals("urlSession")) {
|
||||
CollectionValue collectionValue = entry.getValue().getStringCollectionValueHashMap().get(items[i].getItem());
|
||||
collectionValue.setStat_time(context.window().getEnd() / 1000);
|
||||
jsonStr = JSONObject.toJSONString(collectionValue);
|
||||
} else {
|
||||
TopUrlEntity topUrlEntity = entry.getValue().getStringTopUrlEntityHashMap().get(items[i].getItem());
|
||||
topUrlEntity.setStat_time(context.window().getEnd() / 1000);
|
||||
jsonStr = JSONObject.toJSONString(topUrlEntity);
|
||||
}
|
||||
|
||||
|
||||
collector.collect(jsonStr);
|
||||
|
||||
// String item = items[i].toString();
|
||||
// stringBuilder.append(resultStr);
|
||||
|
||||
// stringBuilder.append("\n");
|
||||
|
||||
|
||||
if (i == TOP_LIMIT)//够条数就结束
|
||||
break;
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -1,57 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import org.apache.flink.api.common.functions.AggregateFunction;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
public class metricsAggregation implements AggregateFunction<Entity, Tuple1<HashMap<String, Entity>>, Tuple1<HashMap<String, Entity>>> {
|
||||
|
||||
private final int topSize;
|
||||
|
||||
public metricsAggregation(int i) {
|
||||
|
||||
this.topSize = i;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Tuple1<HashMap<String, Entity>> createAccumulator() {
|
||||
|
||||
|
||||
return Tuple1.of(new HashMap<String, Entity>());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Tuple1<HashMap<String, Entity>> add(Entity value, Tuple1<HashMap<String, Entity>> accumulator) {
|
||||
|
||||
|
||||
if(accumulator.f0.containsKey(value.getKey_by())) {
|
||||
|
||||
Entity value1= accumulator.f0.get(value.getKey_by());
|
||||
value1.setCommon_c2s_pkt_num(value1.getCommon_c2s_pkt_num() + value.getCommon_c2s_pkt_num());
|
||||
value1.setCommon_s2c_pkt_num(value1.getCommon_s2c_pkt_num() + value.getCommon_s2c_pkt_num());
|
||||
value1.setCommon_c2s_byte_num(value1.getCommon_c2s_byte_num() + value.getCommon_c2s_byte_num());
|
||||
value1.setCommon_s2c_byte_num(value1.getCommon_s2c_byte_num() + value.getCommon_s2c_byte_num());
|
||||
value1.setCommon_sessions(value1.getCommon_sessions() + value.getCommon_sessions());
|
||||
}
|
||||
else {
|
||||
if(accumulator.f0.size()<topSize){
|
||||
accumulator.f0.put(value.getKey_by(),value);
|
||||
}
|
||||
|
||||
}
|
||||
return Tuple1.of(accumulator.f0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Tuple1<HashMap<String, Entity>> getResult(Tuple1<HashMap<String, Entity>> accumulator) {
|
||||
return accumulator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Tuple1<HashMap<String, Entity>> merge(Tuple1<HashMap<String, Entity>> a, Tuple1<HashMap<String, Entity>> b) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -1,18 +1,17 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import com.galaxy.tsg.pojo.transformEntity;
|
||||
import org.apache.flink.api.common.functions.ReduceFunction;
|
||||
|
||||
public class metricsAggregationReduce implements ReduceFunction<Entity> {
|
||||
public class metricsAggregationReduce implements ReduceFunction<transformEntity> {
|
||||
|
||||
@Override
|
||||
public Entity reduce(Entity value1, Entity value2) throws Exception {
|
||||
value1.setCommon_c2s_pkt_num(value1.getCommon_c2s_pkt_num() + value2.getCommon_c2s_pkt_num());
|
||||
value1.setCommon_s2c_pkt_num(value1.getCommon_s2c_pkt_num() + value2.getCommon_s2c_pkt_num());
|
||||
value1.setCommon_c2s_byte_num(value1.getCommon_c2s_byte_num() + value2.getCommon_c2s_byte_num());
|
||||
value1.setCommon_s2c_byte_num(value1.getCommon_s2c_byte_num() + value2.getCommon_s2c_byte_num());
|
||||
value1.setCommon_sessions(value1.getCommon_sessions()+value2.getCommon_sessions());
|
||||
|
||||
public transformEntity reduce(transformEntity value1, transformEntity value2) throws Exception {
|
||||
value1.setOut_pkts(value1.getOut_pkts() + value2.getOut_pkts());
|
||||
value1.setOut_bytes(value1.getOut_bytes() + value2.getOut_bytes());
|
||||
value1.setIn_bytes(value1.getIn_bytes() + value2.getIn_bytes());
|
||||
value1.setIn_pkts(value1.getIn_pkts() + value2.getIn_pkts());
|
||||
value1.setSessions(value1.getSessions()+value2.getSessions());
|
||||
return value1;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,243 +1,193 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.*;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.api.java.tuple.Tuple5;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class metricsCalculate extends ProcessWindowFunction<
|
||||
Entity, // 输入类型
|
||||
ResultEntity, // 输出类型
|
||||
Tuple4<String,Long,String,String>, // 键类型
|
||||
public class metricsCalculate extends ProcessWindowFunction<
|
||||
transformEntity, // 输入类型
|
||||
resultEntity, // 输出类型
|
||||
Tuple5<String, Long, String, String, String>, // 键类型
|
||||
TimeWindow> { // 窗口类型
|
||||
private final int topSize;
|
||||
private final String key;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<SessionResultEntity> sessionOrderEntity ;
|
||||
private PriorityQueue<PacketResultEntity> packetOrderEntity ;
|
||||
private PriorityQueue<ByteResultEntity> byteOrderEntity ;
|
||||
private PriorityQueue<sessionResultEntity> sessionOrderEntity;
|
||||
private PriorityQueue<packetResultEntity> packetOrderEntity;
|
||||
private PriorityQueue<byteResultEntity> byteOrderEntity;
|
||||
|
||||
public metricsCalculate(int i,String key) {
|
||||
private long windowEndTimestamp;
|
||||
|
||||
public metricsCalculate(int i, String key) {
|
||||
this.key = key;
|
||||
this.topSize = i;
|
||||
this.sessionOrderEntity=new PriorityQueue<>();
|
||||
this.packetOrderEntity=new PriorityQueue<>();
|
||||
this.byteOrderEntity=new PriorityQueue<>();
|
||||
this.sessionOrderEntity = new PriorityQueue<>();
|
||||
this.packetOrderEntity = new PriorityQueue<>();
|
||||
this.byteOrderEntity = new PriorityQueue<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(Tuple4<String,Long,String,String> s,
|
||||
public void process(Tuple5<String, Long, String, String, String> s,
|
||||
Context context,
|
||||
Iterable<Entity> elements, Collector<ResultEntity> out) throws Exception {
|
||||
Iterable<transformEntity> elements, Collector<resultEntity> out) throws Exception {
|
||||
|
||||
if (context.window().getEnd() == windowEndTimestamp) {
|
||||
|
||||
|
||||
|
||||
if (elements.iterator().hasNext()) {
|
||||
|
||||
for(Entity objectEntity : elements) {
|
||||
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
sessionOrderEntity.add(enrichessionResult(context.window().getEnd() / 1000, objectEntity));
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
SessionResultEntity res = sessionOrderEntity.peek();
|
||||
if (res.getSession_num() <= objectEntity.getCommon_sessions()) {
|
||||
sessionOrderEntity.poll();
|
||||
sessionOrderEntity.add(enrichessionResult(context.window().getEnd() / 1000, objectEntity));
|
||||
transformEntity objectTransformEntity = elements.iterator().next();
|
||||
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
sessionOrderEntity.add(enrichessionResult(context.window().getEnd(), objectTransformEntity));
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
sessionResultEntity res = sessionOrderEntity.peek();
|
||||
if (res.getSessions() <= objectTransformEntity.getSessions()) {
|
||||
sessionOrderEntity.poll();
|
||||
sessionOrderEntity.add(enrichessionResult(context.window().getEnd(), objectTransformEntity));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if (packetOrderEntity.size() < topSize) {
|
||||
packetOrderEntity.add(enrichPacketResult(context.window().getEnd() / 1000, objectEntity));
|
||||
} else {
|
||||
if (packetOrderEntity.peek() != null) {
|
||||
PacketResultEntity res = packetOrderEntity.peek();
|
||||
if ((res.getS2c_pkt_num() + res.getC2s_pkt_num()) <= (objectEntity.getCommon_s2c_pkt_num() + objectEntity.getCommon_c2s_pkt_num())) {
|
||||
packetOrderEntity.poll();
|
||||
packetOrderEntity.add(enrichPacketResult(context.window().getEnd() / 1000, objectEntity));
|
||||
if (packetOrderEntity.size() < topSize) {
|
||||
packetOrderEntity.add(enrichPacketResult(context.window().getEnd(), objectTransformEntity));
|
||||
} else {
|
||||
if (packetOrderEntity.peek() != null) {
|
||||
packetResultEntity res = packetOrderEntity.peek();
|
||||
if ((res.getIn_pkts() + res.getOut_pkts()) <= (objectTransformEntity.getIn_pkts() + objectTransformEntity.getOut_pkts())) {
|
||||
packetOrderEntity.poll();
|
||||
packetOrderEntity.add(enrichPacketResult(context.window().getEnd() , objectTransformEntity));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (byteOrderEntity.size() < topSize) {
|
||||
byteOrderEntity.add(enrichByteResult(context.window().getEnd() / 1000, objectEntity));
|
||||
} else {
|
||||
if (byteOrderEntity.peek() != null) {
|
||||
ByteResultEntity res = byteOrderEntity.peek();
|
||||
if ((res.getS2c_byte_num() + res.getC2s_byte_num()) <= (objectEntity.getCommon_s2c_byte_num() + objectEntity.getCommon_c2s_byte_num())) {
|
||||
byteOrderEntity.poll();
|
||||
byteOrderEntity.add(enrichByteResult(context.window().getEnd() / 1000, objectEntity));
|
||||
if (byteOrderEntity.size() < topSize) {
|
||||
byteOrderEntity.add(enrichByteResult(context.window().getEnd(), objectTransformEntity));
|
||||
} else {
|
||||
if (byteOrderEntity.peek() != null) {
|
||||
byteResultEntity res = byteOrderEntity.peek();
|
||||
if ((res.getIn_bytes() + res.getOut_bytes()) <= (objectTransformEntity.getIn_bytes() + objectTransformEntity.getOut_bytes())) {
|
||||
byteOrderEntity.poll();
|
||||
byteOrderEntity.add(enrichByteResult(context.window().getEnd(), objectTransformEntity));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
} else {
|
||||
windowEndTimestamp = context.window().getEnd();
|
||||
|
||||
for (com.galaxy.tsg.pojo.sessionResultEntity sessionResultEntity : sessionOrderEntity) {
|
||||
|
||||
resultEntity en = new resultEntity();
|
||||
en.setOrder_by("sessions");
|
||||
en.setStat_time(context.window().getEnd());
|
||||
en.setSessionResultEntity(sessionResultEntity);
|
||||
out.collect(en);
|
||||
}
|
||||
sessionOrderEntity.clear();
|
||||
|
||||
|
||||
for (com.galaxy.tsg.pojo.packetResultEntity packetResultEntity : packetOrderEntity) {
|
||||
resultEntity en = new resultEntity();
|
||||
en.setOrder_by("packets");
|
||||
en.setStat_time(context.window().getEnd());
|
||||
en.setPacketResultEntity(packetResultEntity);
|
||||
out.collect(en);
|
||||
}
|
||||
packetOrderEntity.clear();
|
||||
|
||||
for (com.galaxy.tsg.pojo.byteResultEntity byteResultEntity : byteOrderEntity) {
|
||||
resultEntity en = new resultEntity();
|
||||
en.setOrder_by("bytes");
|
||||
en.setStat_time(context.window().getEnd());
|
||||
en.setByteResultEntity(byteResultEntity);
|
||||
out.collect(en);
|
||||
}
|
||||
byteOrderEntity.clear();
|
||||
|
||||
}
|
||||
|
||||
while (sessionOrderEntity.size() > 0) {
|
||||
SessionResultEntity obj = sessionOrderEntity.peek();
|
||||
//String jsonStr = JSONObject.toJSONString(en);
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("sessions");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setSessionResultEntity(obj);
|
||||
out.collect(en);
|
||||
sessionOrderEntity.remove();
|
||||
}
|
||||
|
||||
|
||||
while (packetOrderEntity.size() > 0) {
|
||||
PacketResultEntity obj = packetOrderEntity.peek();
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("packets");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setPacketResultEntity(obj);
|
||||
out.collect(en);
|
||||
packetOrderEntity.remove();
|
||||
}
|
||||
|
||||
while (byteOrderEntity.size() > 0) {
|
||||
ByteResultEntity obj = byteOrderEntity.peek();
|
||||
ResultEntity en = new ResultEntity();
|
||||
en.setOrder_by("bytes");
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setByteResultEntity(obj);
|
||||
out.collect(en);
|
||||
byteOrderEntity.remove();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public ByteResultEntity enrichByteResult(Long time,Entity objectEntity) {
|
||||
ByteResultEntity en = new ByteResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
public byteResultEntity enrichByteResult(Long time, transformEntity objectTransformEntity) {
|
||||
byteResultEntity en = new byteResultEntity();
|
||||
en.setVsys_id(objectTransformEntity.getVsys_id());
|
||||
en.setTimestamp(time);
|
||||
en.setSessions(objectTransformEntity.getSessions());
|
||||
en.setOrder_by("bytes");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "http_domain":
|
||||
en.setDomain(objectEntity.getHttp_domain());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
en.setServer_ip(objectTransformEntity.getServer_ip());
|
||||
en.setClient_ip(objectTransformEntity.getClient_ip());
|
||||
en.setSubscriber_id(objectTransformEntity.getSubscriber_id());
|
||||
en.setFqdn(objectTransformEntity.getFqdn());
|
||||
en.setExternal_ip(objectTransformEntity.getExternal_ip());
|
||||
en.setInternal_ip(objectTransformEntity.getInternal_ip());
|
||||
en.setDomain(objectTransformEntity.getDomain());
|
||||
en.setDevice_group(objectTransformEntity.getDevice_group());
|
||||
en.setData_center(objectTransformEntity.getData_center());
|
||||
en.setIn_bytes(objectTransformEntity.getIn_bytes());
|
||||
en.setIn_pkts(objectTransformEntity.getIn_pkts());
|
||||
en.setOut_bytes(objectTransformEntity.getOut_bytes());
|
||||
en.setOut_pkts(objectTransformEntity.getOut_pkts());
|
||||
en.setKey_by(objectTransformEntity.getKey_by());
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
}
|
||||
return en;
|
||||
|
||||
}
|
||||
public SessionResultEntity enrichessionResult(Long time,Entity objectEntity){
|
||||
|
||||
SessionResultEntity en =new SessionResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
public sessionResultEntity enrichessionResult(Long time, transformEntity objectTransformEntity) {
|
||||
|
||||
sessionResultEntity en = new sessionResultEntity();
|
||||
en.setVsys_id(objectTransformEntity.getVsys_id());
|
||||
en.setTimestamp(time);
|
||||
en.setSessions(objectTransformEntity.getSessions());
|
||||
en.setOrder_by("sessions");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch(key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "http_domain":
|
||||
en.setDomain(objectEntity.getHttp_domain());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
|
||||
|
||||
}
|
||||
return en;
|
||||
en.setServer_ip(objectTransformEntity.getServer_ip());
|
||||
en.setClient_ip(objectTransformEntity.getClient_ip());
|
||||
en.setSubscriber_id(objectTransformEntity.getSubscriber_id());
|
||||
en.setFqdn(objectTransformEntity.getFqdn());
|
||||
en.setExternal_ip(objectTransformEntity.getExternal_ip());
|
||||
en.setInternal_ip(objectTransformEntity.getInternal_ip());
|
||||
en.setDomain(objectTransformEntity.getDomain());
|
||||
en.setDevice_group(objectTransformEntity.getDevice_group());
|
||||
en.setData_center(objectTransformEntity.getData_center());
|
||||
en.setIn_bytes(objectTransformEntity.getIn_bytes());
|
||||
en.setIn_pkts(objectTransformEntity.getIn_pkts());
|
||||
en.setOut_bytes(objectTransformEntity.getOut_bytes());
|
||||
en.setOut_pkts(objectTransformEntity.getOut_pkts());
|
||||
en.setKey_by(objectTransformEntity.getKey_by());
|
||||
return en;
|
||||
}
|
||||
public PacketResultEntity enrichPacketResult(Long time,Entity objectEntity){
|
||||
PacketResultEntity en =new PacketResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
|
||||
public packetResultEntity enrichPacketResult(Long time, transformEntity objectTransformEntity) {
|
||||
packetResultEntity en = new packetResultEntity();
|
||||
en.setVsys_id(objectTransformEntity.getVsys_id());
|
||||
en.setTimestamp(time);
|
||||
en.setSessions(objectTransformEntity.getSessions());
|
||||
en.setOrder_by("packets");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch(key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
en.setServer_ip(objectTransformEntity.getServer_ip());
|
||||
en.setClient_ip(objectTransformEntity.getClient_ip());
|
||||
en.setSubscriber_id(objectTransformEntity.getSubscriber_id());
|
||||
en.setFqdn(objectTransformEntity.getFqdn());
|
||||
en.setExternal_ip(objectTransformEntity.getExternal_ip());
|
||||
en.setInternal_ip(objectTransformEntity.getInternal_ip());
|
||||
en.setDomain(objectTransformEntity.getDomain());
|
||||
en.setDevice_group(objectTransformEntity.getDevice_group());
|
||||
en.setData_center(objectTransformEntity.getData_center());
|
||||
en.setIn_bytes(objectTransformEntity.getIn_bytes());
|
||||
en.setIn_pkts(objectTransformEntity.getIn_pkts());
|
||||
en.setOut_bytes(objectTransformEntity.getOut_bytes());
|
||||
en.setOut_pkts(objectTransformEntity.getOut_pkts());
|
||||
en.setKey_by(objectTransformEntity.getKey_by());
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
|
||||
}
|
||||
return en;
|
||||
}
|
||||
}
|
||||
@@ -1,43 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson.JSONObject;
|
||||
import com.galaxy.tsg.pojo.AppEntity;
|
||||
import com.galaxy.tsg.pojo.Entity;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
public class metricsCalculateForApp extends ProcessWindowFunction<
|
||||
Entity, // 输入类型
|
||||
String, // 输出类型
|
||||
Tuple4<String,Long,String,String>, // 键类型
|
||||
TimeWindow> { // 窗口类型
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void process(Tuple4<String,Long,String,String> s,
|
||||
Context context,
|
||||
Iterable<Entity> elements, Collector<String> out) throws Exception {
|
||||
|
||||
|
||||
for (Entity objectEntity: elements) {
|
||||
AppEntity en =new AppEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
String jsonStr = JSONObject.toJSONString(en);
|
||||
out.collect(jsonStr);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
@@ -1,70 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.ResultEntity;
|
||||
import com.galaxy.tsg.pojo.TopUrlEntity;
|
||||
import com.galaxy.tsg.pojo.UrlEntity;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class metricsCalculateForUrl extends ProcessWindowFunction<
|
||||
UrlEntity, // 输入类型
|
||||
ResultEntity, // 输出类型
|
||||
Tuple2<String,Long>, // 键类型
|
||||
TimeWindow> { // 窗口类型
|
||||
private final int topSize;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<TopUrlEntity> sessionOrderEntity;
|
||||
|
||||
|
||||
public metricsCalculateForUrl(int i) {
|
||||
this.topSize = i;
|
||||
this.sessionOrderEntity = new PriorityQueue<>();
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(Tuple2 < String, Long > s,
|
||||
Context context,
|
||||
Iterable<UrlEntity> elements, Collector<ResultEntity> out) throws Exception {
|
||||
|
||||
|
||||
for (UrlEntity objectEntity: elements) {
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
TopUrlEntity en = new TopUrlEntity();
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setUrl(objectEntity.getHttp_url());
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
sessionOrderEntity.add(en);
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
TopUrlEntity res = sessionOrderEntity.peek();
|
||||
if (res.getSession_num() <= objectEntity.getCommon_sessions()) {
|
||||
sessionOrderEntity.poll();
|
||||
TopUrlEntity en = new TopUrlEntity();
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setStat_time(context.window().getEnd() / 1000);
|
||||
en.setUrl(objectEntity.getHttp_url());
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
sessionOrderEntity.add(en);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
while (sessionOrderEntity.size() > 0) {
|
||||
TopUrlEntity obj = sessionOrderEntity.peek();
|
||||
ResultEntity resultEntity = new ResultEntity();
|
||||
resultEntity.setOrder_by("sessions");
|
||||
resultEntity.setStat_time(context.window().getEnd() / 1000);
|
||||
resultEntity.setTopUrlEntity(obj);
|
||||
out.collect(resultEntity);
|
||||
sessionOrderEntity.remove();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
@@ -1,110 +0,0 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.galaxy.tsg.pojo.*;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
import org.apache.flink.api.java.tuple.Tuple4;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class metricsCalculateTest extends ProcessWindowFunction<
|
||||
Tuple1<HashMap<String, Entity>>, // 输入类型
|
||||
ResultEntity, // 输出类型
|
||||
Tuple4<String,Long,String,String>, // 键类型
|
||||
TimeWindow> { // 窗口类型
|
||||
private final int topSize;
|
||||
private final String key;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<SessionResultEntity> sessionOrderEntity ;
|
||||
private PriorityQueue<PacketResultEntity> packetOrderEntity ;
|
||||
private PriorityQueue<ByteResultEntity> byteOrderEntity ;
|
||||
|
||||
public metricsCalculateTest(int i, String key) {
|
||||
this.key = key;
|
||||
this.topSize = i;
|
||||
this.sessionOrderEntity=new PriorityQueue<>();
|
||||
this.packetOrderEntity=new PriorityQueue<>();
|
||||
this.byteOrderEntity=new PriorityQueue<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(Tuple4<String,Long,String,String> s,
|
||||
Context context,
|
||||
Iterable<Tuple1<HashMap<String, Entity>>> elements, Collector<ResultEntity> out) throws Exception {
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
for(Tuple1<HashMap<String, Entity>> objectEntity : elements) {
|
||||
|
||||
for (Map.Entry<String, Entity> entry: objectEntity.f0.entrySet()){
|
||||
|
||||
|
||||
|
||||
ByteResultEntity testEntity = enrichByteResult(context.window().getEnd() / 1000,entry.getValue());
|
||||
|
||||
ResultEntity resultEntity = new ResultEntity();
|
||||
resultEntity.setOrder_by("");
|
||||
resultEntity.setByteResultEntity(testEntity);
|
||||
resultEntity.setStat_time(context.window().getEnd() / 1000);
|
||||
out.collect(resultEntity);
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
public ByteResultEntity enrichByteResult(Long time,Entity objectEntity) {
|
||||
ByteResultEntity en = new ByteResultEntity();
|
||||
en.setVsys_id(objectEntity.getCommon_vsys_id());
|
||||
en.setStat_time(time);
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
en.setSession_num(objectEntity.getCommon_sessions());
|
||||
en.setOrder_by("");
|
||||
en.setDevice_group(objectEntity.getCommon_device_group());
|
||||
en.setData_center(objectEntity.getCommon_data_center());
|
||||
en.setS2c_pkt_num(objectEntity.getCommon_s2c_pkt_num());
|
||||
en.setC2s_pkt_num(objectEntity.getCommon_c2s_pkt_num());
|
||||
en.setS2c_byte_num(objectEntity.getCommon_s2c_byte_num());
|
||||
en.setC2s_byte_num(objectEntity.getCommon_c2s_byte_num());
|
||||
switch (key) {
|
||||
case "common_client_ip":
|
||||
en.setSource(objectEntity.getCommon_client_ip());
|
||||
break;
|
||||
case "common_server_ip":
|
||||
en.setDestination(objectEntity.getCommon_server_ip());
|
||||
break;
|
||||
case "common_internal_ip":
|
||||
en.setSource(objectEntity.getCommon_internal_ip());
|
||||
break;
|
||||
case "common_external_ip":
|
||||
en.setDestination(objectEntity.getCommon_external_ip());
|
||||
break;
|
||||
case "http_domain":
|
||||
en.setDomain(objectEntity.getHttp_domain());
|
||||
break;
|
||||
|
||||
case "common_subscriber_id":
|
||||
en.setSubscriber_id(objectEntity.getCommon_subscriber_id());
|
||||
break;
|
||||
|
||||
case "common_app_label":
|
||||
en.setApp_name(objectEntity.getCommon_app_label());
|
||||
break;
|
||||
|
||||
default:
|
||||
|
||||
}
|
||||
return en;
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
260
src/main/java/com/galaxy/tsg/function/topnHotItems.java
Normal file
260
src/main/java/com/galaxy/tsg/function/topnHotItems.java
Normal file
@@ -0,0 +1,260 @@
|
||||
package com.galaxy.tsg.function;
|
||||
|
||||
import com.alibaba.fastjson2.JSONObject;
|
||||
import com.galaxy.tsg.pojo.*;
|
||||
import org.apache.commons.collections.map.LinkedMap;
|
||||
import org.apache.flink.api.java.tuple.Tuple1;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
public class topnHotItems extends KeyedProcessFunction<Tuple1<String>, resultEntity, String> {
|
||||
private final int topSize;
|
||||
// Set<ObjectEntity> allSet = new TreeSet<ObjectEntity>() ;
|
||||
private PriorityQueue<sessionResultEntity> sessionOrderEntity ;
|
||||
private PriorityQueue<packetResultEntity> packetOrderEntity ;
|
||||
private PriorityQueue<byteResultEntity> byteOrderEntity ;
|
||||
|
||||
|
||||
public topnHotItems(int i) {
|
||||
this.topSize = i;
|
||||
|
||||
|
||||
|
||||
}
|
||||
@Override
|
||||
public void open(Configuration parameters) {
|
||||
|
||||
this.sessionOrderEntity=new PriorityQueue<>();
|
||||
this.packetOrderEntity=new PriorityQueue<>();
|
||||
this.byteOrderEntity=new PriorityQueue<>();
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void processElement(resultEntity objectEntity, Context context, Collector<String> collector) {
|
||||
|
||||
|
||||
|
||||
|
||||
switch(objectEntity.getOrder_by()) {
|
||||
case "sessions":
|
||||
|
||||
|
||||
if (sessionOrderEntity.size() < topSize) {
|
||||
sessionOrderEntity.add(objectEntity.getSessionResultEntity());
|
||||
} else {
|
||||
if (sessionOrderEntity.peek() != null) {
|
||||
sessionResultEntity res=sessionOrderEntity.peek();
|
||||
if (res.getSessions() <= objectEntity.getSessionResultEntity().getSessions()) {
|
||||
sessionOrderEntity.poll();
|
||||
sessionOrderEntity.add(objectEntity.getSessionResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
break;
|
||||
case "packets":
|
||||
if (packetOrderEntity.size() < topSize) {
|
||||
packetOrderEntity.add(objectEntity.getPacketResultEntity());
|
||||
} else {
|
||||
if (packetOrderEntity.peek() != null) {
|
||||
packetResultEntity res=packetOrderEntity.peek();
|
||||
if ((res.getIn_pkts()+res.getOut_pkts()) <= (objectEntity.getPacketResultEntity().getIn_pkts()+objectEntity.getPacketResultEntity().getOut_pkts())) {
|
||||
packetOrderEntity.poll();
|
||||
packetOrderEntity.add(objectEntity.getPacketResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
case "bytes":
|
||||
if (byteOrderEntity.size() < topSize) {
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
} else {
|
||||
if (byteOrderEntity.peek() != null) {
|
||||
byteResultEntity res=byteOrderEntity.peek();
|
||||
if ((res.getIn_bytes()+res.getOut_bytes()) <= (objectEntity.getByteResultEntity().getIn_bytes()+objectEntity.getByteResultEntity().getOut_bytes())) {
|
||||
byteOrderEntity.poll();
|
||||
byteOrderEntity.add(objectEntity.getByteResultEntity());
|
||||
}
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
|
||||
}
|
||||
|
||||
//注册 windowEnd+1 的 EventTime Timer, 当触发时,说明收集好了所有 windowEnd的商品数据
|
||||
context.timerService().registerEventTimeTimer(objectEntity.getStat_time() + 1);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void onTimer(long timestamp, OnTimerContext ctx, Collector<String> out) {
|
||||
|
||||
for(sessionResultEntity en : sessionOrderEntity) {
|
||||
|
||||
metricsEntity metricsEntity = new metricsEntity();
|
||||
metricsEntity.setName("sessions_top_" + en.getKey_by());
|
||||
Map tags = new LinkedMap();
|
||||
tags.put("vsys_id", en.getVsys_id());
|
||||
tags.put("device_id", en.getDevice_id());
|
||||
tags.put("device_group", en.getDevice_group());
|
||||
tags.put("data_center", en.getData_center());
|
||||
switch (en.getKey_by()) {
|
||||
case "client_ip":
|
||||
tags.put("client_ip", en.getClient_ip());
|
||||
break;
|
||||
case "server_ip":
|
||||
tags.put("server_ip", en.getServer_ip());
|
||||
break;
|
||||
case "internal_ip":
|
||||
tags.put("internal_ip", en.getInternal_ip());
|
||||
break;
|
||||
case "external_ip":
|
||||
tags.put("external_ip", en.getExternal_ip());
|
||||
break;
|
||||
case "server_domain":
|
||||
tags.put("domain", en.getDomain());
|
||||
break;
|
||||
case "subscriber_id":
|
||||
tags.put("subscriber_id", en.getSubscriber_id());
|
||||
break;
|
||||
case "server_fqdn":
|
||||
tags.put("fqdn", en.getFqdn());
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
Map fields = new LinkedMap();
|
||||
fields.put("sessions", en.getSessions());
|
||||
fields.put("out_bytes", en.getOut_bytes());
|
||||
fields.put("in_bytes", en.getIn_bytes());
|
||||
fields.put("out_pkts", en.getOut_pkts());
|
||||
fields.put("in_pkts", en.getIn_pkts());
|
||||
metricsEntity.setFields(fields);
|
||||
metricsEntity.setTags(tags);
|
||||
metricsEntity.setTimestamp(en.getTimestamp());
|
||||
String jsonStr = JSONObject.toJSONString(metricsEntity);
|
||||
out.collect(jsonStr);
|
||||
|
||||
|
||||
}
|
||||
|
||||
for(packetResultEntity en : packetOrderEntity){
|
||||
|
||||
|
||||
metricsEntity metricsEntity = new metricsEntity();
|
||||
metricsEntity.setName("packets_top_" + en.getKey_by());
|
||||
Map tags = new LinkedMap();
|
||||
tags.put("vsys_id", en.getVsys_id());
|
||||
tags.put("device_id", en.getDevice_id());
|
||||
|
||||
tags.put("device_group", en.getDevice_group());
|
||||
tags.put("data_center", en.getData_center());
|
||||
switch (en.getKey_by()) {
|
||||
case "client_ip":
|
||||
tags.put("client_ip", en.getClient_ip());
|
||||
break;
|
||||
case "server_ip":
|
||||
tags.put("server_ip", en.getServer_ip());
|
||||
break;
|
||||
case "internal_ip":
|
||||
tags.put("internal_ip", en.getInternal_ip());
|
||||
break;
|
||||
case "external_ip":
|
||||
tags.put("external_ip", en.getExternal_ip());
|
||||
break;
|
||||
case "server_domain":
|
||||
tags.put("domain", en.getDomain());
|
||||
break;
|
||||
case "subscriber_id":
|
||||
tags.put("subscriber_id", en.getSubscriber_id());
|
||||
break;
|
||||
case "server_fqdn":
|
||||
tags.put("fqdn", en.getFqdn());
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
Map fields = new LinkedMap();
|
||||
fields.put("sessions", en.getSessions());
|
||||
fields.put("out_bytes", en.getOut_bytes());
|
||||
fields.put("in_bytes", en.getIn_bytes());
|
||||
fields.put("out_pkts", en.getOut_pkts());
|
||||
fields.put("in_pkts", en.getIn_pkts());
|
||||
metricsEntity.setFields(fields);
|
||||
metricsEntity.setTags(tags);
|
||||
metricsEntity.setTimestamp(en.getTimestamp());
|
||||
String jsonStr = JSONObject.toJSONString(metricsEntity);
|
||||
out.collect(jsonStr);
|
||||
|
||||
|
||||
}
|
||||
for(byteResultEntity en : byteOrderEntity){
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
metricsEntity metricsEntity = new metricsEntity();
|
||||
metricsEntity.setName("bytes_top_" + en.getKey_by());
|
||||
Map tags = new LinkedMap();
|
||||
tags.put("vsys_id", en.getVsys_id());
|
||||
tags.put("device_id", en.getDevice_id());
|
||||
tags.put("device_group", en.getDevice_group());
|
||||
tags.put("data_center", en.getData_center());
|
||||
switch (en.getKey_by()) {
|
||||
case "client_ip":
|
||||
tags.put("client_ip", en.getClient_ip());
|
||||
break;
|
||||
case "server_ip":
|
||||
tags.put("server_ip", en.getServer_ip());
|
||||
break;
|
||||
case "internal_ip":
|
||||
tags.put("internal_ip", en.getInternal_ip());
|
||||
break;
|
||||
case "external_ip":
|
||||
tags.put("external_ip", en.getExternal_ip());
|
||||
break;
|
||||
case "server_domain":
|
||||
tags.put("domain", en.getDomain());
|
||||
break;
|
||||
case "subscriber_id":
|
||||
tags.put("subscriber_id", en.getSubscriber_id());
|
||||
break;
|
||||
case "server_fqdn":
|
||||
tags.put("fqdn", en.getFqdn());
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
Map fields = new LinkedMap();
|
||||
fields.put("sessions", en.getSessions());
|
||||
fields.put("out_bytes", en.getOut_bytes());
|
||||
fields.put("in_bytes", en.getIn_bytes());
|
||||
fields.put("out_pkts", en.getOut_pkts());
|
||||
fields.put("in_pkts", en.getIn_pkts());
|
||||
metricsEntity.setFields(fields);
|
||||
metricsEntity.setTags(tags);
|
||||
metricsEntity.setTimestamp(en.getTimestamp());
|
||||
String jsonStr = JSONObject.toJSONString(metricsEntity);
|
||||
out.collect(jsonStr);
|
||||
|
||||
}
|
||||
|
||||
sessionOrderEntity.clear();
|
||||
packetOrderEntity.clear();
|
||||
byteOrderEntity.clear();
|
||||
|
||||
|
||||
|
||||
}
|
||||
}
|
||||
@@ -1,95 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class AppEntity {
|
||||
|
||||
private Long vsys_id ;
|
||||
private Long session_num ;
|
||||
private Long c2s_pkt_num ;
|
||||
private Long s2c_pkt_num;
|
||||
private Long c2s_byte_num ;
|
||||
private Long s2c_byte_num ;
|
||||
private String device_group;
|
||||
private String data_center;
|
||||
private Long stat_time;
|
||||
private String app_name;
|
||||
|
||||
public Long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(Long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public Long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(Long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
public Long getC2s_pkt_num() {
|
||||
return c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setC2s_pkt_num(Long c2s_pkt_num) {
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
}
|
||||
|
||||
public Long getS2c_pkt_num() {
|
||||
return s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setS2c_pkt_num(Long s2c_pkt_num) {
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
}
|
||||
|
||||
public Long getC2s_byte_num() {
|
||||
return c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setC2s_byte_num(Long c2s_byte_num) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
}
|
||||
|
||||
public Long getS2c_byte_num() {
|
||||
return s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setS2c_byte_num(Long s2c_byte_num) {
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
public String getApp_name() {
|
||||
return app_name;
|
||||
}
|
||||
|
||||
public void setApp_name(String app_name) {
|
||||
this.app_name = app_name;
|
||||
}
|
||||
}
|
||||
@@ -1,152 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class ByteResultEntity implements Comparable<ByteResultEntity> {
|
||||
|
||||
|
||||
private String source ;
|
||||
private Long vsys_id ;
|
||||
private Long session_num ;
|
||||
private Long c2s_pkt_num ;
|
||||
private Long s2c_pkt_num;
|
||||
private Long c2s_byte_num ;
|
||||
private Long s2c_byte_num ;
|
||||
private String order_by;
|
||||
private String device_group;
|
||||
private String data_center;
|
||||
private Long stat_time;
|
||||
private String destination ;
|
||||
private String domain;
|
||||
private String subscriber_id;
|
||||
private String app_name;
|
||||
|
||||
public String getApp_name() {
|
||||
return app_name;
|
||||
}
|
||||
|
||||
public void setApp_name(String app_name) {
|
||||
this.app_name = app_name;
|
||||
}
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public String getDestination() {
|
||||
return destination;
|
||||
}
|
||||
|
||||
public void setDestination(String destination) {
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
public String getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
public void setSource(String source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
public Long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(Long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public Long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(Long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
public Long getC2s_pkt_num() {
|
||||
return c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setC2s_pkt_num(Long c2s_pkt_num) {
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
}
|
||||
|
||||
public Long getS2c_pkt_num() {
|
||||
return s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setS2c_pkt_num(Long s2c_pkt_num) {
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
}
|
||||
|
||||
public Long getC2s_byte_num() {
|
||||
return c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setC2s_byte_num(Long c2s_byte_num) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
}
|
||||
|
||||
public Long getS2c_byte_num() {
|
||||
return s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setS2c_byte_num(Long s2c_byte_num) {
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
}
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ByteResultEntity per) {
|
||||
if(this.session_num>=per.session_num){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,211 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
public class Entity implements Serializable {
|
||||
|
||||
public int ifError;
|
||||
public String common_client_ip ;
|
||||
public String common_app_label ;
|
||||
public long common_recv_time ;
|
||||
|
||||
public String common_schema_type ;
|
||||
public String common_server_ip ;
|
||||
public String http_host ;
|
||||
public String http_domain ;
|
||||
public long common_vsys_id ;
|
||||
public String common_device_group ;
|
||||
public String common_data_center;
|
||||
public String common_l4_protocol;
|
||||
public String common_internal_ip;
|
||||
public String common_external_ip;
|
||||
public String common_subscriber_id;
|
||||
public long common_sessions;
|
||||
public long common_c2s_pkt_num;
|
||||
public long common_s2c_pkt_num;
|
||||
public long common_c2s_byte_num ;
|
||||
public long common_s2c_byte_num ;
|
||||
public String key_by;
|
||||
|
||||
public int count;
|
||||
|
||||
public Entity() {
|
||||
}
|
||||
|
||||
public int getCount() {
|
||||
return count;
|
||||
}
|
||||
|
||||
public void setCount(int count) {
|
||||
this.count = count;
|
||||
}
|
||||
|
||||
public String getKey_by() {
|
||||
return key_by;
|
||||
}
|
||||
|
||||
public void setKey_by(String key_by) {
|
||||
this.key_by = key_by;
|
||||
}
|
||||
|
||||
public int getIfError() {
|
||||
return ifError;
|
||||
}
|
||||
|
||||
public void setIfError(int ifError) {
|
||||
this.ifError = ifError;
|
||||
}
|
||||
|
||||
public String getCommon_client_ip() {
|
||||
return common_client_ip;
|
||||
}
|
||||
|
||||
public void setCommon_client_ip(String common_client_ip) {
|
||||
this.common_client_ip = common_client_ip;
|
||||
}
|
||||
|
||||
public String getCommon_app_label() {
|
||||
return common_app_label;
|
||||
}
|
||||
|
||||
public void setCommon_app_label(String common_app_label) {
|
||||
this.common_app_label = common_app_label;
|
||||
}
|
||||
|
||||
public long getCommon_recv_time() {
|
||||
return common_recv_time;
|
||||
}
|
||||
|
||||
public void setCommon_recv_time(long common_recv_time) {
|
||||
this.common_recv_time = common_recv_time;
|
||||
}
|
||||
|
||||
public String getCommon_schema_type() {
|
||||
return common_schema_type;
|
||||
}
|
||||
|
||||
public void setCommon_schema_type(String common_schema_type) {
|
||||
this.common_schema_type = common_schema_type;
|
||||
}
|
||||
|
||||
public String getCommon_server_ip() {
|
||||
return common_server_ip;
|
||||
}
|
||||
|
||||
public void setCommon_server_ip(String common_server_ip) {
|
||||
this.common_server_ip = common_server_ip;
|
||||
}
|
||||
|
||||
public String getHttp_host() {
|
||||
return http_host;
|
||||
}
|
||||
|
||||
public void setHttp_host(String http_host) {
|
||||
this.http_host = http_host;
|
||||
}
|
||||
|
||||
public String getHttp_domain() {
|
||||
return http_domain;
|
||||
}
|
||||
|
||||
public void setHttp_domain(String http_domain) {
|
||||
this.http_domain = http_domain;
|
||||
}
|
||||
|
||||
public long getCommon_vsys_id() {
|
||||
return common_vsys_id;
|
||||
}
|
||||
|
||||
public void setCommon_vsys_id(long common_vsys_id) {
|
||||
this.common_vsys_id = common_vsys_id;
|
||||
}
|
||||
|
||||
public String getCommon_device_group() {
|
||||
return common_device_group;
|
||||
}
|
||||
|
||||
public void setCommon_device_group(String common_device_group) {
|
||||
this.common_device_group = common_device_group;
|
||||
}
|
||||
|
||||
|
||||
public String getCommon_data_center() {
|
||||
return common_data_center;
|
||||
}
|
||||
|
||||
public void setCommon_data_center(String common_data_center) {
|
||||
this.common_data_center = common_data_center;
|
||||
}
|
||||
|
||||
public String getCommon_l4_protocol() {
|
||||
return common_l4_protocol;
|
||||
}
|
||||
|
||||
public void setCommon_l4_protocol(String common_l4_protocol) {
|
||||
this.common_l4_protocol = common_l4_protocol;
|
||||
}
|
||||
|
||||
public String getCommon_internal_ip() {
|
||||
return common_internal_ip;
|
||||
}
|
||||
|
||||
public void setCommon_internal_ip(String common_internal_ip) {
|
||||
this.common_internal_ip = common_internal_ip;
|
||||
}
|
||||
|
||||
public String getCommon_external_ip() {
|
||||
return common_external_ip;
|
||||
}
|
||||
|
||||
public void setCommon_external_ip(String common_external_ip) {
|
||||
this.common_external_ip = common_external_ip;
|
||||
}
|
||||
|
||||
public String getCommon_subscriber_id() {
|
||||
return common_subscriber_id;
|
||||
}
|
||||
|
||||
public void setCommon_subscriber_id(String common_subscriber_id) {
|
||||
this.common_subscriber_id = common_subscriber_id;
|
||||
}
|
||||
|
||||
public long getCommon_sessions() {
|
||||
return common_sessions;
|
||||
}
|
||||
|
||||
public void setCommon_sessions(long common_sessions) {
|
||||
this.common_sessions = common_sessions;
|
||||
}
|
||||
|
||||
public long getCommon_c2s_pkt_num() {
|
||||
return common_c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setCommon_c2s_pkt_num(long common_c2s_pkt_num) {
|
||||
this.common_c2s_pkt_num = common_c2s_pkt_num;
|
||||
}
|
||||
|
||||
public long getCommon_s2c_pkt_num() {
|
||||
return common_s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setCommon_s2c_pkt_num(long common_s2c_pkt_num) {
|
||||
this.common_s2c_pkt_num = common_s2c_pkt_num;
|
||||
}
|
||||
|
||||
public long getCommon_c2s_byte_num() {
|
||||
return common_c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setCommon_c2s_byte_num(long common_c2s_byte_num) {
|
||||
this.common_c2s_byte_num = common_c2s_byte_num;
|
||||
}
|
||||
|
||||
public long getCommon_s2c_byte_num() {
|
||||
return common_s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setCommon_s2c_byte_num(long common_s2c_byte_num) {
|
||||
this.common_s2c_byte_num = common_s2c_byte_num;
|
||||
}
|
||||
}
|
||||
@@ -1,163 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class PacketResultEntity implements Comparable<PacketResultEntity>, Cloneable {
|
||||
|
||||
|
||||
private String source ;
|
||||
private Long vsys_id ;
|
||||
private Long session_num ;
|
||||
private Long c2s_pkt_num ;
|
||||
private Long s2c_pkt_num;
|
||||
private Long c2s_byte_num ;
|
||||
private Long s2c_byte_num ;
|
||||
private String order_by;
|
||||
private String device_group;
|
||||
private String data_center;
|
||||
private Long stat_time;
|
||||
|
||||
private String destination ;
|
||||
private String domain;
|
||||
private String subscriber_id;
|
||||
private String app_name;
|
||||
|
||||
public String getApp_name() {
|
||||
return app_name;
|
||||
}
|
||||
|
||||
public void setApp_name(String app_name) {
|
||||
this.app_name = app_name;
|
||||
}
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
public String getDestination() {
|
||||
return destination;
|
||||
}
|
||||
|
||||
public void setDestination(String destination) {
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
|
||||
public String getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
public void setSource(String source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
public Long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(Long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public Long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(Long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
public Long getC2s_pkt_num() {
|
||||
return c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setC2s_pkt_num(Long c2s_pkt_num) {
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
}
|
||||
|
||||
public Long getS2c_pkt_num() {
|
||||
return s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setS2c_pkt_num(Long s2c_pkt_num) {
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
}
|
||||
|
||||
public Long getC2s_byte_num() {
|
||||
return c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setC2s_byte_num(Long c2s_byte_num) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
}
|
||||
|
||||
public Long getS2c_byte_num() {
|
||||
return s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setS2c_byte_num(Long s2c_byte_num) {
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
}
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(PacketResultEntity per) {
|
||||
if(this.session_num>=per.session_num){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/* @Override public Object clone() {
|
||||
PacketResultEntity obj;
|
||||
try {
|
||||
obj = (PacketResultEntity) super.clone();
|
||||
} catch (CloneNotSupportedException e) {
|
||||
obj = new PacketResultEntity(this.source,this.vsys_id,this.session_num,this.c2s_pkt_num,this.s2c_pkt_num,this.c2s_byte_num,
|
||||
this.s2c_byte_num,this.order_by,this.device_group,this.data_center,this.stat_time);
|
||||
}
|
||||
return obj;
|
||||
}
|
||||
*/
|
||||
|
||||
}
|
||||
@@ -1,63 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class ResultEntity {
|
||||
|
||||
|
||||
|
||||
private String order_by;
|
||||
private Long stat_time;
|
||||
|
||||
private SessionResultEntity sessionResultEntity;
|
||||
private PacketResultEntity packetResultEntity;
|
||||
private ByteResultEntity byteResultEntity;
|
||||
private TopUrlEntity topUrlEntity;
|
||||
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public TopUrlEntity getTopUrlEntity() {
|
||||
return topUrlEntity;
|
||||
}
|
||||
|
||||
public void setTopUrlEntity(TopUrlEntity topUrlEntity) {
|
||||
this.topUrlEntity = topUrlEntity;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
public SessionResultEntity getSessionResultEntity() {
|
||||
return sessionResultEntity;
|
||||
}
|
||||
|
||||
public void setSessionResultEntity(SessionResultEntity sessionResultEntity) {
|
||||
this.sessionResultEntity = sessionResultEntity;
|
||||
}
|
||||
|
||||
public PacketResultEntity getPacketResultEntity() {
|
||||
return packetResultEntity;
|
||||
}
|
||||
|
||||
public void setPacketResultEntity(PacketResultEntity packetResultEntity) {
|
||||
this.packetResultEntity = packetResultEntity;
|
||||
}
|
||||
|
||||
public ByteResultEntity getByteResultEntity() {
|
||||
return byteResultEntity;
|
||||
}
|
||||
|
||||
public void setByteResultEntity(ByteResultEntity byteResultEntity) {
|
||||
this.byteResultEntity = byteResultEntity;
|
||||
}
|
||||
}
|
||||
@@ -1,151 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class SessionResultEntity implements Comparable<SessionResultEntity> {
|
||||
|
||||
|
||||
private String source ;
|
||||
private Long vsys_id ;
|
||||
private Long session_num ;
|
||||
private Long c2s_pkt_num ;
|
||||
private Long s2c_pkt_num;
|
||||
private Long c2s_byte_num ;
|
||||
private Long s2c_byte_num ;
|
||||
private String order_by;
|
||||
private String device_group;
|
||||
private String data_center;
|
||||
private Long stat_time;
|
||||
private String destination ;
|
||||
private String domain;
|
||||
private String subscriber_id;
|
||||
private String app_name;
|
||||
|
||||
public String getApp_name() {
|
||||
return app_name;
|
||||
}
|
||||
|
||||
public void setApp_name(String app_name) {
|
||||
this.app_name = app_name;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
public String getDestination() {
|
||||
return destination;
|
||||
}
|
||||
|
||||
public void setDestination(String destination) {
|
||||
this.destination = destination;
|
||||
}
|
||||
|
||||
public String getSource() {
|
||||
return source;
|
||||
}
|
||||
|
||||
public void setSource(String source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
public Long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(Long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public Long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(Long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
public Long getC2s_pkt_num() {
|
||||
return c2s_pkt_num;
|
||||
}
|
||||
|
||||
public void setC2s_pkt_num(Long c2s_pkt_num) {
|
||||
this.c2s_pkt_num = c2s_pkt_num;
|
||||
}
|
||||
|
||||
public Long getS2c_pkt_num() {
|
||||
return s2c_pkt_num;
|
||||
}
|
||||
|
||||
public void setS2c_pkt_num(Long s2c_pkt_num) {
|
||||
this.s2c_pkt_num = s2c_pkt_num;
|
||||
}
|
||||
|
||||
public Long getC2s_byte_num() {
|
||||
return c2s_byte_num;
|
||||
}
|
||||
|
||||
public void setC2s_byte_num(Long c2s_byte_num) {
|
||||
this.c2s_byte_num = c2s_byte_num;
|
||||
}
|
||||
|
||||
public Long getS2c_byte_num() {
|
||||
return s2c_byte_num;
|
||||
}
|
||||
|
||||
public void setS2c_byte_num(Long s2c_byte_num) {
|
||||
this.s2c_byte_num = s2c_byte_num;
|
||||
}
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(SessionResultEntity per) {
|
||||
if(this.session_num>=per.session_num){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,54 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class TopUrlEntity implements Comparable<TopUrlEntity>{
|
||||
|
||||
|
||||
public long stat_time ;
|
||||
|
||||
public String url ;
|
||||
|
||||
public long vsys_id ;
|
||||
|
||||
public long session_num;
|
||||
|
||||
public long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
public String getUrl() {
|
||||
return url;
|
||||
}
|
||||
|
||||
public void setUrl(String url) {
|
||||
this.url = url;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public long getSession_num() {
|
||||
return session_num;
|
||||
}
|
||||
|
||||
public void setSession_num(long session_num) {
|
||||
this.session_num = session_num;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(TopUrlEntity per) {
|
||||
if(this.session_num>=per.session_num){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,55 +0,0 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class UrlEntity {
|
||||
|
||||
|
||||
public long common_recv_time ;
|
||||
|
||||
public String http_url ;
|
||||
|
||||
public long common_vsys_id ;
|
||||
|
||||
public long common_sessions;
|
||||
|
||||
public int ifError;
|
||||
|
||||
public int getIfError() {
|
||||
return ifError;
|
||||
}
|
||||
|
||||
public void setIfError(int ifError) {
|
||||
this.ifError = ifError;
|
||||
}
|
||||
|
||||
public long getCommon_recv_time() {
|
||||
return common_recv_time;
|
||||
}
|
||||
|
||||
public void setCommon_recv_time(long common_recv_time) {
|
||||
this.common_recv_time = common_recv_time;
|
||||
}
|
||||
|
||||
public String getHttp_url() {
|
||||
return http_url;
|
||||
}
|
||||
|
||||
public void setHttp_url(String http_url) {
|
||||
this.http_url = http_url;
|
||||
}
|
||||
|
||||
public long getCommon_vsys_id() {
|
||||
return common_vsys_id;
|
||||
}
|
||||
|
||||
public void setCommon_vsys_id(long common_vsys_id) {
|
||||
this.common_vsys_id = common_vsys_id;
|
||||
}
|
||||
|
||||
public long getCommon_sessions() {
|
||||
return common_sessions;
|
||||
}
|
||||
|
||||
public void setCommon_sessions(long common_sessions) {
|
||||
this.common_sessions = common_sessions;
|
||||
}
|
||||
}
|
||||
204
src/main/java/com/galaxy/tsg/pojo/byteResultEntity.java
Normal file
204
src/main/java/com/galaxy/tsg/pojo/byteResultEntity.java
Normal file
@@ -0,0 +1,204 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class byteResultEntity implements Comparable<byteResultEntity> {
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
private long timestamp ;
|
||||
|
||||
|
||||
private String fqdn;
|
||||
private String server_ip ;
|
||||
private String domain ;
|
||||
private long vsys_id ;
|
||||
private String device_group ;
|
||||
private String client_ip ;
|
||||
private String device_id;
|
||||
private String data_center;
|
||||
private String internal_ip;
|
||||
private String external_ip;
|
||||
private String subscriber_id;
|
||||
private long sessions;
|
||||
private long out_bytes;
|
||||
private long in_bytes;
|
||||
private long out_pkts ;
|
||||
private long in_pkts ;
|
||||
private String key_by;
|
||||
private String l4_protocol;
|
||||
private String order_by;
|
||||
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public long getTimestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
public void setTimestamp(long timestamp) {
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
|
||||
public String getFqdn() {
|
||||
return fqdn;
|
||||
}
|
||||
|
||||
public void setFqdn(String fqdn) {
|
||||
this.fqdn = fqdn;
|
||||
}
|
||||
|
||||
public String getServer_ip() {
|
||||
return server_ip;
|
||||
}
|
||||
|
||||
public void setServer_ip(String server_ip) {
|
||||
this.server_ip = server_ip;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getClient_ip() {
|
||||
return client_ip;
|
||||
}
|
||||
|
||||
public void setClient_ip(String client_ip) {
|
||||
this.client_ip = client_ip;
|
||||
}
|
||||
|
||||
public String getDevice_id() {
|
||||
return device_id;
|
||||
}
|
||||
|
||||
public void setDevice_id(String device_id) {
|
||||
this.device_id = device_id;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public String getInternal_ip() {
|
||||
return internal_ip;
|
||||
}
|
||||
|
||||
public void setInternal_ip(String internal_ip) {
|
||||
this.internal_ip = internal_ip;
|
||||
}
|
||||
|
||||
public String getExternal_ip() {
|
||||
return external_ip;
|
||||
}
|
||||
|
||||
public void setExternal_ip(String external_ip) {
|
||||
this.external_ip = external_ip;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public long getSessions() {
|
||||
return sessions;
|
||||
}
|
||||
|
||||
public void setSessions(long sessions) {
|
||||
this.sessions = sessions;
|
||||
}
|
||||
|
||||
public long getOut_bytes() {
|
||||
return out_bytes;
|
||||
}
|
||||
|
||||
public void setOut_bytes(long out_bytes) {
|
||||
this.out_bytes = out_bytes;
|
||||
}
|
||||
|
||||
public long getIn_bytes() {
|
||||
return in_bytes;
|
||||
}
|
||||
|
||||
public void setIn_bytes(long in_bytes) {
|
||||
this.in_bytes = in_bytes;
|
||||
}
|
||||
|
||||
public long getOut_pkts() {
|
||||
return out_pkts;
|
||||
}
|
||||
|
||||
public void setOut_pkts(long out_pkts) {
|
||||
this.out_pkts = out_pkts;
|
||||
}
|
||||
|
||||
public long getIn_pkts() {
|
||||
return in_pkts;
|
||||
}
|
||||
|
||||
public void setIn_pkts(long in_pkts) {
|
||||
this.in_pkts = in_pkts;
|
||||
}
|
||||
|
||||
public String getKey_by() {
|
||||
return key_by;
|
||||
}
|
||||
|
||||
public void setKey_by(String key_by) {
|
||||
this.key_by = key_by;
|
||||
}
|
||||
|
||||
public String getL4_protocol() {
|
||||
return l4_protocol;
|
||||
}
|
||||
|
||||
public void setL4_protocol(String l4_protocol) {
|
||||
this.l4_protocol = l4_protocol;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(byteResultEntity per) {
|
||||
if((this.out_bytes+this.in_bytes)>=(per.out_bytes+per.in_bytes)){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
213
src/main/java/com/galaxy/tsg/pojo/packetResultEntity.java
Normal file
213
src/main/java/com/galaxy/tsg/pojo/packetResultEntity.java
Normal file
@@ -0,0 +1,213 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class packetResultEntity implements Comparable<packetResultEntity>, Cloneable {
|
||||
|
||||
|
||||
private long timestamp ;
|
||||
|
||||
|
||||
private String fqdn;
|
||||
private String server_ip ;
|
||||
private String domain ;
|
||||
private long vsys_id ;
|
||||
private String device_group ;
|
||||
private String client_ip ;
|
||||
private String device_id;
|
||||
private String data_center;
|
||||
private String internal_ip;
|
||||
private String external_ip;
|
||||
private String subscriber_id;
|
||||
private long sessions;
|
||||
private long out_bytes;
|
||||
private long in_bytes;
|
||||
private long out_pkts ;
|
||||
private long in_pkts ;
|
||||
private String key_by;
|
||||
private String l4_protocol;
|
||||
|
||||
private String order_by;
|
||||
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public long getTimestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
public void setTimestamp(long timestamp) {
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
|
||||
public String getFqdn() {
|
||||
return fqdn;
|
||||
}
|
||||
|
||||
public void setFqdn(String fqdn) {
|
||||
this.fqdn = fqdn;
|
||||
}
|
||||
|
||||
public String getServer_ip() {
|
||||
return server_ip;
|
||||
}
|
||||
|
||||
public void setServer_ip(String server_ip) {
|
||||
this.server_ip = server_ip;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getClient_ip() {
|
||||
return client_ip;
|
||||
}
|
||||
|
||||
public void setClient_ip(String client_ip) {
|
||||
this.client_ip = client_ip;
|
||||
}
|
||||
|
||||
public String getDevice_id() {
|
||||
return device_id;
|
||||
}
|
||||
|
||||
public void setDevice_id(String device_id) {
|
||||
this.device_id = device_id;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public String getInternal_ip() {
|
||||
return internal_ip;
|
||||
}
|
||||
|
||||
public void setInternal_ip(String internal_ip) {
|
||||
this.internal_ip = internal_ip;
|
||||
}
|
||||
|
||||
public String getExternal_ip() {
|
||||
return external_ip;
|
||||
}
|
||||
|
||||
public void setExternal_ip(String external_ip) {
|
||||
this.external_ip = external_ip;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public long getSessions() {
|
||||
return sessions;
|
||||
}
|
||||
|
||||
public void setSessions(long sessions) {
|
||||
this.sessions = sessions;
|
||||
}
|
||||
|
||||
public long getOut_bytes() {
|
||||
return out_bytes;
|
||||
}
|
||||
|
||||
public void setOut_bytes(long out_bytes) {
|
||||
this.out_bytes = out_bytes;
|
||||
}
|
||||
|
||||
public long getIn_bytes() {
|
||||
return in_bytes;
|
||||
}
|
||||
|
||||
public void setIn_bytes(long in_bytes) {
|
||||
this.in_bytes = in_bytes;
|
||||
}
|
||||
|
||||
public long getOut_pkts() {
|
||||
return out_pkts;
|
||||
}
|
||||
|
||||
public void setOut_pkts(long out_pkts) {
|
||||
this.out_pkts = out_pkts;
|
||||
}
|
||||
|
||||
public long getIn_pkts() {
|
||||
return in_pkts;
|
||||
}
|
||||
|
||||
public void setIn_pkts(long in_pkts) {
|
||||
this.in_pkts = in_pkts;
|
||||
}
|
||||
|
||||
public String getKey_by() {
|
||||
return key_by;
|
||||
}
|
||||
|
||||
public void setKey_by(String key_by) {
|
||||
this.key_by = key_by;
|
||||
}
|
||||
|
||||
public String getL4_protocol() {
|
||||
return l4_protocol;
|
||||
}
|
||||
|
||||
public void setL4_protocol(String l4_protocol) {
|
||||
this.l4_protocol = l4_protocol;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(packetResultEntity per) {
|
||||
if((this.out_pkts+this.in_pkts)>=(per.out_pkts+per.in_pkts)){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/* @Override public Object clone() {
|
||||
PacketResultEntity obj;
|
||||
try {
|
||||
obj = (PacketResultEntity) super.clone();
|
||||
} catch (CloneNotSupportedException e) {
|
||||
obj = new PacketResultEntity(this.source,this.vsys_id,this.session_num,this.c2s_pkt_num,this.s2c_pkt_num,this.c2s_byte_num,
|
||||
this.s2c_byte_num,this.order_by,this.device_group,this.data_center,this.stat_time);
|
||||
}
|
||||
return obj;
|
||||
}
|
||||
*/
|
||||
|
||||
}
|
||||
56
src/main/java/com/galaxy/tsg/pojo/resultEntity.java
Normal file
56
src/main/java/com/galaxy/tsg/pojo/resultEntity.java
Normal file
@@ -0,0 +1,56 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class resultEntity {
|
||||
|
||||
|
||||
|
||||
private String order_by;
|
||||
private Long stat_time;
|
||||
|
||||
private com.galaxy.tsg.pojo.sessionResultEntity sessionResultEntity;
|
||||
private com.galaxy.tsg.pojo.packetResultEntity packetResultEntity;
|
||||
private com.galaxy.tsg.pojo.byteResultEntity byteResultEntity;
|
||||
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public Long getStat_time() {
|
||||
return stat_time;
|
||||
}
|
||||
|
||||
public void setStat_time(Long stat_time) {
|
||||
this.stat_time = stat_time;
|
||||
}
|
||||
|
||||
public com.galaxy.tsg.pojo.sessionResultEntity getSessionResultEntity() {
|
||||
return sessionResultEntity;
|
||||
}
|
||||
|
||||
public void setSessionResultEntity(com.galaxy.tsg.pojo.sessionResultEntity sessionResultEntity) {
|
||||
this.sessionResultEntity = sessionResultEntity;
|
||||
}
|
||||
|
||||
public com.galaxy.tsg.pojo.packetResultEntity getPacketResultEntity() {
|
||||
return packetResultEntity;
|
||||
}
|
||||
|
||||
public void setPacketResultEntity(com.galaxy.tsg.pojo.packetResultEntity packetResultEntity) {
|
||||
this.packetResultEntity = packetResultEntity;
|
||||
}
|
||||
|
||||
public com.galaxy.tsg.pojo.byteResultEntity getByteResultEntity() {
|
||||
return byteResultEntity;
|
||||
}
|
||||
|
||||
public void setByteResultEntity(com.galaxy.tsg.pojo.byteResultEntity byteResultEntity) {
|
||||
this.byteResultEntity = byteResultEntity;
|
||||
}
|
||||
}
|
||||
202
src/main/java/com/galaxy/tsg/pojo/sessionResultEntity.java
Normal file
202
src/main/java/com/galaxy/tsg/pojo/sessionResultEntity.java
Normal file
@@ -0,0 +1,202 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
public class sessionResultEntity implements Comparable<sessionResultEntity> {
|
||||
|
||||
|
||||
|
||||
private long timestamp ;
|
||||
|
||||
|
||||
private String fqdn;
|
||||
private String server_ip ;
|
||||
private String domain ;
|
||||
private long vsys_id ;
|
||||
private String device_group ;
|
||||
private String client_ip ;
|
||||
private String device_id;
|
||||
private String data_center;
|
||||
private String internal_ip;
|
||||
private String external_ip;
|
||||
private String subscriber_id;
|
||||
private long sessions;
|
||||
private long out_bytes;
|
||||
private long in_bytes;
|
||||
private long out_pkts ;
|
||||
private long in_pkts ;
|
||||
private String key_by;
|
||||
private String l4_protocol;
|
||||
private String order_by;
|
||||
|
||||
|
||||
public String getOrder_by() {
|
||||
return order_by;
|
||||
}
|
||||
|
||||
public void setOrder_by(String order_by) {
|
||||
this.order_by = order_by;
|
||||
}
|
||||
|
||||
public long getTimestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
public void setTimestamp(long timestamp) {
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
|
||||
public String getFqdn() {
|
||||
return fqdn;
|
||||
}
|
||||
|
||||
public void setFqdn(String fqdn) {
|
||||
this.fqdn = fqdn;
|
||||
}
|
||||
|
||||
public String getServer_ip() {
|
||||
return server_ip;
|
||||
}
|
||||
|
||||
public void setServer_ip(String server_ip) {
|
||||
this.server_ip = server_ip;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getClient_ip() {
|
||||
return client_ip;
|
||||
}
|
||||
|
||||
public void setClient_ip(String client_ip) {
|
||||
this.client_ip = client_ip;
|
||||
}
|
||||
|
||||
public String getDevice_id() {
|
||||
return device_id;
|
||||
}
|
||||
|
||||
public void setDevice_id(String device_id) {
|
||||
this.device_id = device_id;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public String getInternal_ip() {
|
||||
return internal_ip;
|
||||
}
|
||||
|
||||
public void setInternal_ip(String internal_ip) {
|
||||
this.internal_ip = internal_ip;
|
||||
}
|
||||
|
||||
public String getExternal_ip() {
|
||||
return external_ip;
|
||||
}
|
||||
|
||||
public void setExternal_ip(String external_ip) {
|
||||
this.external_ip = external_ip;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public long getSessions() {
|
||||
return sessions;
|
||||
}
|
||||
|
||||
public void setSessions(long sessions) {
|
||||
this.sessions = sessions;
|
||||
}
|
||||
|
||||
public long getOut_bytes() {
|
||||
return out_bytes;
|
||||
}
|
||||
|
||||
public void setOut_bytes(long out_bytes) {
|
||||
this.out_bytes = out_bytes;
|
||||
}
|
||||
|
||||
public long getIn_bytes() {
|
||||
return in_bytes;
|
||||
}
|
||||
|
||||
public void setIn_bytes(long in_bytes) {
|
||||
this.in_bytes = in_bytes;
|
||||
}
|
||||
|
||||
public long getOut_pkts() {
|
||||
return out_pkts;
|
||||
}
|
||||
|
||||
public void setOut_pkts(long out_pkts) {
|
||||
this.out_pkts = out_pkts;
|
||||
}
|
||||
|
||||
public long getIn_pkts() {
|
||||
return in_pkts;
|
||||
}
|
||||
|
||||
public void setIn_pkts(long in_pkts) {
|
||||
this.in_pkts = in_pkts;
|
||||
}
|
||||
|
||||
public String getKey_by() {
|
||||
return key_by;
|
||||
}
|
||||
|
||||
public void setKey_by(String key_by) {
|
||||
this.key_by = key_by;
|
||||
}
|
||||
|
||||
public String getL4_protocol() {
|
||||
return l4_protocol;
|
||||
}
|
||||
|
||||
public void setL4_protocol(String l4_protocol) {
|
||||
this.l4_protocol = l4_protocol;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(sessionResultEntity per) {
|
||||
if(this.sessions>=per.sessions){
|
||||
return 1 ;
|
||||
}else{
|
||||
return -1 ;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
194
src/main/java/com/galaxy/tsg/pojo/transformEntity.java
Normal file
194
src/main/java/com/galaxy/tsg/pojo/transformEntity.java
Normal file
@@ -0,0 +1,194 @@
|
||||
package com.galaxy.tsg.pojo;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
public class transformEntity implements Serializable {
|
||||
|
||||
public int ifError;
|
||||
public long timestamp ;
|
||||
|
||||
|
||||
public String fqdn;
|
||||
public String server_ip ;
|
||||
public String domain ;
|
||||
public long vsys_id ;
|
||||
public String device_group ;
|
||||
public String client_ip ;
|
||||
public String device_id;
|
||||
public String data_center;
|
||||
public String internal_ip;
|
||||
public String external_ip;
|
||||
public String subscriber_id;
|
||||
public long sessions;
|
||||
public long out_bytes;
|
||||
public long in_bytes;
|
||||
public long out_pkts ;
|
||||
public long in_pkts ;
|
||||
public String key_by;
|
||||
public String l4_protocol;
|
||||
|
||||
public transformEntity() {
|
||||
}
|
||||
|
||||
public String getL4_protocol() {
|
||||
return l4_protocol;
|
||||
}
|
||||
|
||||
public void setL4_protocol(String l4_protocol) {
|
||||
this.l4_protocol = l4_protocol;
|
||||
}
|
||||
|
||||
public int getIfError() {
|
||||
return ifError;
|
||||
}
|
||||
|
||||
public void setIfError(int ifError) {
|
||||
this.ifError = ifError;
|
||||
}
|
||||
|
||||
public long getTimestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
public void setTimestamp(long timestamp) {
|
||||
this.timestamp = timestamp;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public String getFqdn() {
|
||||
return fqdn;
|
||||
}
|
||||
|
||||
public void setFqdn(String fqdn) {
|
||||
this.fqdn = fqdn;
|
||||
}
|
||||
|
||||
public String getServer_ip() {
|
||||
return server_ip;
|
||||
}
|
||||
|
||||
public void setServer_ip(String server_ip) {
|
||||
this.server_ip = server_ip;
|
||||
}
|
||||
|
||||
public String getDomain() {
|
||||
return domain;
|
||||
}
|
||||
|
||||
public void setDomain(String domain) {
|
||||
this.domain = domain;
|
||||
}
|
||||
|
||||
public long getVsys_id() {
|
||||
return vsys_id;
|
||||
}
|
||||
|
||||
public void setVsys_id(long vsys_id) {
|
||||
this.vsys_id = vsys_id;
|
||||
}
|
||||
|
||||
public String getDevice_group() {
|
||||
return device_group;
|
||||
}
|
||||
|
||||
public void setDevice_group(String device_group) {
|
||||
this.device_group = device_group;
|
||||
}
|
||||
|
||||
public String getClient_ip() {
|
||||
return client_ip;
|
||||
}
|
||||
|
||||
public void setClient_ip(String client_ip) {
|
||||
this.client_ip = client_ip;
|
||||
}
|
||||
|
||||
public String getDevice_id() {
|
||||
return device_id;
|
||||
}
|
||||
|
||||
public void setDevice_id(String device_id) {
|
||||
this.device_id = device_id;
|
||||
}
|
||||
|
||||
public String getData_center() {
|
||||
return data_center;
|
||||
}
|
||||
|
||||
public void setData_center(String data_center) {
|
||||
this.data_center = data_center;
|
||||
}
|
||||
|
||||
public String getInternal_ip() {
|
||||
return internal_ip;
|
||||
}
|
||||
|
||||
public void setInternal_ip(String internal_ip) {
|
||||
this.internal_ip = internal_ip;
|
||||
}
|
||||
|
||||
public String getExternal_ip() {
|
||||
return external_ip;
|
||||
}
|
||||
|
||||
public void setExternal_ip(String external_ip) {
|
||||
this.external_ip = external_ip;
|
||||
}
|
||||
|
||||
public String getSubscriber_id() {
|
||||
return subscriber_id;
|
||||
}
|
||||
|
||||
public void setSubscriber_id(String subscriber_id) {
|
||||
this.subscriber_id = subscriber_id;
|
||||
}
|
||||
|
||||
public long getSessions() {
|
||||
return sessions;
|
||||
}
|
||||
|
||||
public void setSessions(long sessions) {
|
||||
this.sessions = sessions;
|
||||
}
|
||||
|
||||
public long getOut_bytes() {
|
||||
return out_bytes;
|
||||
}
|
||||
|
||||
public void setOut_bytes(long out_bytes) {
|
||||
this.out_bytes = out_bytes;
|
||||
}
|
||||
|
||||
public long getIn_bytes() {
|
||||
return in_bytes;
|
||||
}
|
||||
|
||||
public void setIn_bytes(long in_bytes) {
|
||||
this.in_bytes = in_bytes;
|
||||
}
|
||||
|
||||
public long getOut_pkts() {
|
||||
return out_pkts;
|
||||
}
|
||||
|
||||
public void setOut_pkts(long out_pkts) {
|
||||
this.out_pkts = out_pkts;
|
||||
}
|
||||
|
||||
public long getIn_pkts() {
|
||||
return in_pkts;
|
||||
}
|
||||
|
||||
public void setIn_pkts(long in_pkts) {
|
||||
this.in_pkts = in_pkts;
|
||||
}
|
||||
|
||||
public String getKey_by() {
|
||||
return key_by;
|
||||
}
|
||||
|
||||
public void setKey_by(String key_by) {
|
||||
this.key_by = key_by;
|
||||
}
|
||||
}
|
||||
@@ -11,7 +11,7 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
|
||||
public class KafkaUtils {
|
||||
public class kafkaUtils {
|
||||
|
||||
|
||||
public static Properties getKafkaSourceProperty() {
|
||||
@@ -22,21 +22,21 @@ public class KafkaUtils {
|
||||
properties.setProperty("max.poll.records", commonConfig.KAFKA_CONSUMER_MAX_POLL_RECORD);
|
||||
properties.setProperty("max.partition.fetch.bytes", commonConfig.KAFKA_CONSUMER_MAX_PARTITION_FETCH_BYTES);
|
||||
|
||||
switch (commonConfig.KAFKA_SECURITY) {
|
||||
switch (commonConfig.KAFKA_CONSUMER_SECURITY) {
|
||||
case 1:
|
||||
properties.put("security.protocol", "SSL");
|
||||
properties.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
|
||||
properties.put("ssl.keystore.location", commonConfig.TOOLS_LIBRARY + "keystore.jks");
|
||||
properties.put("ssl.keystore.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.truststore.location", commonConfig.TOOLS_LIBRARY + "truststore.jks");
|
||||
properties.put("ssl.truststore.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.key.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.keystore.location", commonConfig.TOOLS_CONSUMER_LIBRARY + "keystore.jks");
|
||||
properties.put("ssl.keystore.password", commonConfig.KAFKA_CONSUMER_PIN);
|
||||
properties.put("ssl.truststore.location", commonConfig.TOOLS_CONSUMER_LIBRARY + "truststore.jks");
|
||||
properties.put("ssl.truststore.password", commonConfig.KAFKA_CONSUMER_PIN);
|
||||
properties.put("ssl.key.password", commonConfig.KAFKA_CONSUMER_PIN);
|
||||
break;
|
||||
case 2:
|
||||
properties.put("security.protocol", "SASL_PLAINTEXT");
|
||||
properties.put("sasl.mechanism", "PLAIN");
|
||||
properties.put("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username="
|
||||
+ commonConfig.KAFKA_USER + " password=" + commonConfig.KAFKA_PIN + ";");
|
||||
+ commonConfig.KAFKA_CONSUMER_USER + " password=" + commonConfig.KAFKA_CONSUMER_PIN + ";");
|
||||
break;
|
||||
default:
|
||||
}
|
||||
@@ -57,21 +57,21 @@ public class KafkaUtils {
|
||||
properties.put("max.request.size", commonConfig.KAFKA_PRODUCER_MAX_REQUEST_SIZE);
|
||||
properties.put("compression.type", commonConfig.KAFKA_PRODUCER_COMPRESSION_TYPE);
|
||||
|
||||
switch (commonConfig.KAFKA_SECURITY) {
|
||||
switch (commonConfig.KAFKA_PRODUCER_SECURITY) {
|
||||
case 1:
|
||||
properties.put("security.protocol", "SSL");
|
||||
properties.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "");
|
||||
properties.put("ssl.keystore.location", commonConfig.TOOLS_LIBRARY + "keystore.jks");
|
||||
properties.put("ssl.keystore.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.truststore.location", commonConfig.TOOLS_LIBRARY + "truststore.jks");
|
||||
properties.put("ssl.truststore.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.key.password", commonConfig.KAFKA_PIN);
|
||||
properties.put("ssl.keystore.location", commonConfig.TOOLS_PRODUCER_LIBRARY + "keystore.jks");
|
||||
properties.put("ssl.keystore.password", commonConfig.KAFKA_PRODUCER_PIN);
|
||||
properties.put("ssl.truststore.location", commonConfig.TOOLS_PRODUCER_LIBRARY + "truststore.jks");
|
||||
properties.put("ssl.truststore.password", commonConfig.KAFKA_PRODUCER_PIN);
|
||||
properties.put("ssl.key.password", commonConfig.KAFKA_PRODUCER_PIN);
|
||||
break;
|
||||
case 2:
|
||||
properties.put("security.protocol", "SASL_PLAINTEXT");
|
||||
properties.put("sasl.mechanism", "PLAIN");
|
||||
properties.put("sasl.jaas.config", "org.apache.kafka.common.security.plain.PlainLoginModule required username="
|
||||
+ commonConfig.KAFKA_USER + " password=" + commonConfig.KAFKA_PIN + ";");
|
||||
+ commonConfig.KAFKA_PRODUCER_USER + " password=" + commonConfig.KAFKA_PRODUCER_PIN + ";");
|
||||
break;
|
||||
default:
|
||||
}
|
||||
@@ -1,40 +1,83 @@
|
||||
#--------------------------------Kafka<6B><61><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>Ϣ------------------------------#
|
||||
#kafka<6B>ĵ<EFBFBD>ַ<EFBFBD><D6B7>Ϣ
|
||||
kafka.consumer.broker=192.168.44.11:9092
|
||||
kafka.consumer.group.id =vpn-1206-27
|
||||
kafka.consumer.broker=192.168.44.12:9094
|
||||
|
||||
#kafka <20><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>topic
|
||||
kafka.consumer.topic=SESSION-RECORD-COMPLETED
|
||||
kafka.consumer.parallelism=5
|
||||
kafka.consumer.max.poll.records=3000
|
||||
kafka.consumer.session.timeout.ms=60000
|
||||
kafka.consumer.max.partition.fetch.bytes=31457280
|
||||
kafka.consumer.topurl.parallelism=5
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
kafka.consumer.group.id=topn-metrics-job-20230501
|
||||
|
||||
#--------------------------------Kafka<6B><61><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>Ϣ------------------------------#
|
||||
#kafka<6B>ĵ<EFBFBD>ַ<EFBFBD><D6B7>Ϣ
|
||||
kafka_producer_broker=192.168.44.12:9094
|
||||
|
||||
kafka.producer.topic=TRAFFIC-TOP-METRICS
|
||||
|
||||
#--------------------------------topology<67><79><EFBFBD><EFBFBD>------------------------------#
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
job.name=TOPN-METRICS-JOB
|
||||
|
||||
#source<63><65><EFBFBD>ж<EFBFBD>
|
||||
kafka.consumer.parallelism=1
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ж<EFBFBD>
|
||||
task.parallelism=5
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD>ӳٵȴ<D9B5>ʱ<EFBFBD>䵥λ<E4B5A5><CEBB>
|
||||
watermark.time=100
|
||||
#top<6F><70><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
top.limit=100
|
||||
task.parallelism=1
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ж<EFBFBD>
|
||||
orderby.parallelism=1
|
||||
|
||||
#<23><><EFBFBD>Ⲣ<EFBFBD>жȣ<D0B6>ͨ<EFBFBD><CDA8><EFBFBD><EFBFBD><EFBFBD><EFBFBD>orderby.parallelism
|
||||
sink.parallelism=1
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD>ӳٵȴ<D9B5>ʱ<EFBFBD>䵥λ<E4B5A5><CEBB>
|
||||
watermark.time=60
|
||||
|
||||
#top<6F><70><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
top.limit=10000
|
||||
|
||||
url.top.limit=100
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ʱ<EFBFBD>䵥λ<E4B5A5><CEBB><EFBFBD><EFBFBD>
|
||||
window.time.minute=5
|
||||
#ÿ<><C3BF><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ʱ<EFBFBD>䵥λ<E4B5A5><CEBB><EFBFBD><EFBFBD>
|
||||
#kafka<EFBFBD>Ƿ<EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ȫ<EFBFBD><EFBFBD>֤ 0<><30><EFBFBD><EFBFBD><EFBFBD><EFBFBD> 1SSL 2 SASL
|
||||
kafka.security=0
|
||||
|
||||
#--------------------------------Kafka<EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>------------------------------#
|
||||
#kafka source poll
|
||||
kafka.consumer.max.poll.records=3000
|
||||
|
||||
#kafka source connection timeout
|
||||
kafka.consumer.session.timeout.ms=60000
|
||||
|
||||
#kafka source poll bytes
|
||||
kafka.consumer.max.partition.fetch.bytes=31457280
|
||||
|
||||
#kafka<6B><61><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>Ƿ<EFBFBD><C7B7><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ȫ<EFBFBD><C8AB>֤ 0<><30><EFBFBD><EFBFBD><EFBFBD><EFBFBD> 1SSL 2 SASL
|
||||
kafka.consumer.security=2
|
||||
|
||||
#kafka SASL<53><4C>֤<EFBFBD>û<EFBFBD><C3BB><EFBFBD>
|
||||
kafka.user=admin
|
||||
kafka.consumer.user=admin
|
||||
|
||||
#kafka SASL<53><4C>SSL<53><4C>֤<EFBFBD><D6A4><EFBFBD><EFBFBD>
|
||||
kafka.pin=galaxy2019
|
||||
kafka.consumer.pin=galaxy2019
|
||||
|
||||
#1SSL<53><4C>Ҫ
|
||||
tools.library=D:\\K18-Phase2\\tsgSpace\\dat\\tsg\\
|
||||
tools.consumer.library=/home/bigdata/topology/dat/
|
||||
|
||||
#kafka<6B><61><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>Ƿ<EFBFBD><C7B7><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ȫ<EFBFBD><C8AB>֤ 0<><30><EFBFBD><EFBFBD><EFBFBD><EFBFBD> 1SSL 2 SASL
|
||||
kafka.producer.security=2
|
||||
|
||||
#kafka SASL<53><4C>֤<EFBFBD>û<EFBFBD><C3BB><EFBFBD>
|
||||
kafka.producer.user=admin
|
||||
|
||||
#kafka SASL<53><4C>SSL<53><4C>֤<EFBFBD><D6A4><EFBFBD><EFBFBD>
|
||||
kafka.producer.pin=galaxy2019
|
||||
|
||||
#1SSL<53><4C>Ҫ
|
||||
tools.producer.library=/home/bigdata/topology/dat/
|
||||
|
||||
#producer<65><72><EFBFBD>ԵĴ<D4B5><C4B4><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
kafka.producer.retries=0
|
||||
|
||||
#<23><><EFBFBD>ĺ<EFBFBD><C4BA><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>˵һ<CBB5><D2BB>Batch<63><68><EFBFBD><EFBFBD><EFBFBD><EFBFBD>֮<EFBFBD><D6AE><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ã<EFBFBD><C3A3><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>Batch<63><68>û<EFBFBD><C3BB>д<EFBFBD><D0B4><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>뷢<EFBFBD>ͳ<EFBFBD>ȥ<EFBFBD><C8A5>
|
||||
kafka.producer.linger.ms=10
|
||||
kafka.producer.linger.ms=1
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD>ڳ<EFBFBD>ʱ֮ǰδ<C7B0>յ<EFBFBD><D5B5><EFBFBD>Ӧ<EFBFBD><D3A6><EFBFBD>ͻ<EFBFBD><CDBB>˽<EFBFBD><CBBD>ڱ<EFBFBD>Ҫʱ<D2AA><CAB1><EFBFBD>·<EFBFBD><C2B7><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>
|
||||
kafka.producer.request.timeout.ms=30000
|
||||
@@ -43,17 +86,10 @@ kafka.producer.request.timeout.ms=30000
|
||||
kafka.producer.batch.size=262144
|
||||
|
||||
#Producer<65><72><EFBFBD><EFBFBD><EFBFBD>ڻ<EFBFBD><DABB><EFBFBD><EFBFBD><EFBFBD>Ϣ<EFBFBD>Ļ<EFBFBD><C4BB><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>С
|
||||
#128M
|
||||
kafka.producer.buffer.memory=134217728
|
||||
|
||||
#<23><><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>ÿ<EFBFBD>η<EFBFBD><CEB7><EFBFBD>Kafka<6B><61><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD><EFBFBD>С,Ĭ<><C4AC>1048576
|
||||
#10M
|
||||
kafka.producer.max.request.size=10485760
|
||||
|
||||
#<23><><EFBFBD><EFBFBD>kafkaѹ<61><D1B9><EFBFBD><EFBFBD><EFBFBD>ͣ<EFBFBD>Ĭ<EFBFBD>ϲ<EFBFBD><CFB2><EFBFBD><EFBFBD><EFBFBD>
|
||||
kafka.producer.compression.type=none
|
||||
|
||||
kafka_producer_broker=192.168.44.12:9092
|
||||
|
||||
|
||||
tmp.test.type=3
|
||||
Reference in New Issue
Block a user