|
@@ -6,16 +6,13 @@ import com.aliyun.odps.account.Account;
|
|
|
import com.aliyun.odps.account.AliyunAccount;
|
|
|
import com.aliyun.odps.data.Record;
|
|
|
import com.aliyun.odps.task.SQLTask;
|
|
|
-import com.tencent.ads.model.DailyReportsGetListStruct;
|
|
|
import com.tencent.ads.model.HourlyReportsGetListStruct;
|
|
|
import flink.zanxiangnet.ad.monitoring.maxcompute.MaxComputeLog;
|
|
|
-import flink.zanxiangnet.ad.monitoring.maxcompute.sink.TunnelBatchSink;
|
|
|
-import flink.zanxiangnet.ad.monitoring.pojo.dto.AdDataOfDayDTO;
|
|
|
-import flink.zanxiangnet.ad.monitoring.pojo.dto.AdStatOfDayODSDTO;
|
|
|
import flink.zanxiangnet.ad.monitoring.pojo.entity.*;
|
|
|
import flink.zanxiangnet.ad.monitoring.pojo.properties.ApplicationProperties;
|
|
|
import flink.zanxiangnet.ad.monitoring.pojo.properties.KafkaProperties;
|
|
|
import flink.zanxiangnet.ad.monitoring.pojo.dto.AdDataOfMinuteDTO;
|
|
|
+import flink.zanxiangnet.ad.monitoring.process.AdMinuteDWDProcess;
|
|
|
import flink.zanxiangnet.ad.monitoring.sink.TunnelBatchStreamSink;
|
|
|
import flink.zanxiangnet.ad.monitoring.stream.KeyedBatchStream;
|
|
|
import flink.zanxiangnet.ad.monitoring.util.DateUtil;
|
|
@@ -23,7 +20,6 @@ import flink.zanxiangnet.ad.monitoring.util.JsonUtil;
|
|
|
import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.flink.api.common.eventtime.*;
|
|
|
import org.apache.flink.api.common.serialization.SimpleStringSchema;
|
|
|
-import org.apache.flink.api.java.functions.KeySelector;
|
|
|
import org.apache.flink.configuration.Configuration;
|
|
|
import org.apache.flink.connector.kafka.source.KafkaSource;
|
|
|
import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer;
|
|
@@ -35,6 +31,8 @@ import org.apache.flink.streaming.api.functions.ProcessFunction;
|
|
|
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
|
|
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
|
|
|
import org.apache.flink.streaming.api.windowing.time.Time;
|
|
|
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
|
|
|
+import org.apache.flink.streaming.api.windowing.triggers.TriggerResult;
|
|
|
import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
|
|
|
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
|
|
import org.apache.flink.util.Collector;
|
|
@@ -130,129 +128,63 @@ public class AdStatJob {
|
|
|
DataStream<AdDataOfMinuteODS> adMinuteODSStream = adODSStream.getSideOutput(adMinuteStreamTag);
|
|
|
// 写入原始表
|
|
|
// adMinuteODSStream.addSink(new TunnelBatchSink<>(AdDataOfMinuteODS.class, 36000L, 64000L, 3));
|
|
|
- new KeyedBatchStream<>("adMinuteODSStream", adMinuteODSStream.keyBy(AdDataOfMinuteODS::getStatDay), 4000L, 60 * 1000L)
|
|
|
- .toBatch().addSink(new TunnelBatchStreamSink<>(AdDataOfMinuteODS.class));
|
|
|
+ new KeyedBatchStream<>("adMinuteODSStream", adMinuteODSStream.keyBy(AdDataOfMinuteODS::getStatDay), 4000L, 2 * 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdDataOfMinuteODS.class));
|
|
|
|
|
|
- adMinuteODSStream
|
|
|
+ SingleOutputStreamOperator<AdStatOfMinuteDWD> adMinuteDWDStream = adMinuteODSStream
|
|
|
// 打水印,允许数据延迟 6分钟,同时指定时间流
|
|
|
- .assignTimestampsAndWatermarks(WatermarkStrategy.<AdDataOfMinuteODS>forBoundedOutOfOrderness(Duration.ofMinutes(6L))
|
|
|
+ .assignTimestampsAndWatermarks(WatermarkStrategy.<AdDataOfMinuteODS>forBoundedOutOfOrderness(Duration.ofMinutes(1L))
|
|
|
.withTimestampAssigner((SerializableTimestampAssigner<AdDataOfMinuteODS>) (adODS, l) -> adODS.getStatTime()))
|
|
|
.keyBy(AdDataOfMinuteODS::getAdId)
|
|
|
// 开一个 5分钟的滚动窗口
|
|
|
.window(TumblingEventTimeWindows.of(Time.minutes(5L)))
|
|
|
- .process(new ProcessWindowFunction<AdDataOfMinuteODS, AdStatOfMinuteDWD, Long, TimeWindow>() {
|
|
|
- // 对象锁,防止MaxCompute的 Tunnel对象多次初始化
|
|
|
- private final String DUMMY_LOCK = "DUMMY_LOCK";
|
|
|
- private Odps odps;
|
|
|
- private long lastClearTime = System.currentTimeMillis();
|
|
|
- private final Map<Long, String> lastAdDayQueryDay = new ConcurrentHashMap<>();
|
|
|
- // 历史的天数据
|
|
|
- private final Map<Long, AdStatOfDayDWD> historyDayData = new ConcurrentHashMap<>();
|
|
|
- // 之前聚合的昨天的数据
|
|
|
- private final Map<Long, Map<String, AdStatOfMinuteDWD>> historyMinuteData = new ConcurrentHashMap<>();
|
|
|
- // 前 5分钟聚合的数据
|
|
|
- private final Map<Long, Map<String, AdStatOfMinuteDWD>> lastReduceData = new ConcurrentHashMap<>();
|
|
|
-
|
|
|
- private void clearData() {
|
|
|
- lastAdDayQueryDay.entrySet().removeIf(entry -> DateUtil.parseLocalDate(entry.getValue()).compareTo(LocalDate.now().minusDays(3L)) <= 0);
|
|
|
- historyDayData.entrySet().removeIf(entry -> DateUtil.parseLocalDate(entry.getValue().getStatDay()).compareTo(LocalDate.now().minusDays(50L)) <= 0);
|
|
|
- historyMinuteData.forEach((key, value) -> {
|
|
|
- if (value != null) {
|
|
|
- value.entrySet().removeIf(entry2 -> DateUtil.parseLocalDate(entry2.getKey()).compareTo(LocalDate.now().minusDays(3L)) <= 0);
|
|
|
- }
|
|
|
- });
|
|
|
- historyMinuteData.entrySet().removeIf(entry -> entry.getValue() == null || entry.getValue().isEmpty());
|
|
|
-
|
|
|
- lastReduceData.forEach((key, value) -> {
|
|
|
- if (value != null) {
|
|
|
- value.entrySet().removeIf(entry -> DateUtil.parseOfMinuteReduce(entry.getKey()).compareTo(LocalDateTime.now().minusHours(1)) <= 0);
|
|
|
- }
|
|
|
- });
|
|
|
- lastReduceData.entrySet().removeIf(entry -> entry.getValue() == null || entry.getValue().isEmpty());
|
|
|
+ .trigger(new Trigger<AdDataOfMinuteODS, TimeWindow>() {
|
|
|
+ @Override
|
|
|
+ public TriggerResult onElement(AdDataOfMinuteODS adDataOfMinuteODS, long time, TimeWindow timeWindow, TriggerContext triggerContext) throws Exception {
|
|
|
+ System.out.println("eventTime:" + DateUtil.formatLocalDateTime(DateUtil.milliToLocalDateTime(time)) +
|
|
|
+ " | 水印时间:" + DateUtil.formatLocalDateTime(DateUtil.milliToLocalDateTime(triggerContext.getCurrentWatermark())));
|
|
|
+ if (timeWindow.maxTimestamp() <= triggerContext.getCurrentWatermark()) {
|
|
|
+ // 到了窗口的最大生命周期
|
|
|
+ return TriggerResult.FIRE_AND_PURGE;
|
|
|
+ }
|
|
|
+ triggerContext.registerEventTimeTimer(timeWindow.maxTimestamp());
|
|
|
+ if (adDataOfMinuteODS.getHour() == DateUtil.milliToLocalDateTime(time).getHour()) {
|
|
|
+ return TriggerResult.FIRE;
|
|
|
+ }
|
|
|
+ return TriggerResult.CONTINUE;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void open(Configuration conf) {
|
|
|
- Map<String, String> params = getRuntimeContext()
|
|
|
- .getExecutionConfig()
|
|
|
- .getGlobalJobParameters()
|
|
|
- .toMap();
|
|
|
- Account account = new AliyunAccount(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_ID),
|
|
|
- params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_KEY));
|
|
|
- odps = new Odps(account);
|
|
|
- odps.getRestClient().setRetryLogger(new MaxComputeLog());
|
|
|
- odps.setEndpoint(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_ENDPOINT));
|
|
|
- odps.setDefaultProject(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_PROJECT_NAME));
|
|
|
+ public TriggerResult onProcessingTime(long time, TimeWindow timeWindow, TriggerContext triggerContext) throws Exception {
|
|
|
+ return TriggerResult.CONTINUE;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void process(Long elementCount, ProcessWindowFunction<AdDataOfMinuteODS, AdStatOfMinuteDWD, Long, TimeWindow>.Context context,
|
|
|
- Iterable<AdDataOfMinuteODS> iterable, Collector<AdStatOfMinuteDWD> collector) throws Exception {
|
|
|
- LocalDateTime beginTime = DateUtil.milliToLocalDateTime(context.window().getStart());
|
|
|
- String statDay = DateUtil.formatLocalDate(beginTime.toLocalDate());
|
|
|
- String reduceTimeKey = DateUtil.formatOfMinuteReduce(beginTime);
|
|
|
- int hour = beginTime.getHour();
|
|
|
- long now = System.currentTimeMillis();
|
|
|
- AdDataOfMinuteODS statODS = null;
|
|
|
- List<AdDataOfMinuteODS> adDataOfMinuteODSList = new ArrayList<>(24);
|
|
|
- for (AdDataOfMinuteODS adDataOfMinuteODS : iterable) {
|
|
|
- adDataOfMinuteODSList.add(adDataOfMinuteODS);
|
|
|
- if (adDataOfMinuteODS.getHour() != hour) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- statODS = adDataOfMinuteODS;
|
|
|
- break;
|
|
|
- }
|
|
|
- if (statODS == null) {
|
|
|
- return;
|
|
|
- }
|
|
|
+ public TriggerResult onEventTime(long time, TimeWindow timeWindow, TriggerContext triggerContext) throws Exception {
|
|
|
+ return time == timeWindow.maxTimestamp() ? TriggerResult.FIRE : TriggerResult.CONTINUE;
|
|
|
+ }
|
|
|
|
|
|
- // 每 12小时清理一次数据
|
|
|
- if (now - lastClearTime > 12 * 60 * 60 * 1000) {
|
|
|
- synchronized (DUMMY_LOCK) {
|
|
|
- if (now - lastClearTime > 12 * 60 * 60 * 1000) {
|
|
|
- clearData();
|
|
|
- lastClearTime = now;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- Long adId = statODS.getAdId();
|
|
|
-
|
|
|
- // 昨天聚合的数据
|
|
|
- Map<String, AdStatOfMinuteDWD> historyMinuteMapping = historyMinuteData.computeIfAbsent(adId, key -> new HashMap<>());
|
|
|
- AdStatOfMinuteDWD yesterdayMinuteDWD = historyMinuteMapping.get(DateUtil.formatLocalDate(beginTime.minusDays(1L).toLocalDate()));
|
|
|
- // 之前的数据
|
|
|
- String lastQueryDay = lastAdDayQueryDay.get(adId);
|
|
|
- if (lastQueryDay == null || !lastQueryDay.equals(statDay)) {
|
|
|
- // 往前找 30天
|
|
|
- LocalDate endDay = beginTime.minusDays(2L).toLocalDate(), beginDay = endDay.minusDays(30L);
|
|
|
- Instance instance = SQLTask.run(odps, "SELECT * FROM ad_stat_of_day_dwd WHERE stat_day >= \"" + DateUtil.formatLocalDate(beginDay) + "\" AND stat_day <= \"" + endDay + "\" AND ad_id = " + adId + ";");
|
|
|
- instance.waitForSuccess();
|
|
|
- List<Record> records = SQLTask.getResult(instance);
|
|
|
- List<AdStatOfDayDWD> temp = records.stream().map(AdStatOfDayDWD::byMaxCompute).sorted((val1, val2) -> {
|
|
|
- if (val1.getStatDay().equals(val2.getStatDay())) {
|
|
|
- return new Long(val1.getCreateTime().getTime() - val2.getCreateTime().getTime()).intValue();
|
|
|
- }
|
|
|
- return DateUtil.parseLocalDate(val1.getStatDay()).compareTo(DateUtil.parseLocalDate(val2.getStatDay()));
|
|
|
- }).collect(Collectors.toList());
|
|
|
- historyDayData.put(adId, temp.isEmpty() ? null : temp.get(temp.size() - 1));
|
|
|
- lastAdDayQueryDay.put(adId, statDay);
|
|
|
- }
|
|
|
- AdStatOfDayDWD beforeYesterdayDayDWD = historyDayData.get(adId);
|
|
|
- Map<String, AdStatOfMinuteDWD> lastReduceMapping = lastReduceData.computeIfAbsent(adId, key -> new HashMap<>());
|
|
|
- // 聚合当天的全部数据
|
|
|
- AdStatOfMinuteDWD nowAdStat = AdStatOfMinuteDWD.reduce(beforeYesterdayDayDWD, yesterdayMinuteDWD, adDataOfMinuteODSList, statODS, lastReduceMapping.get(DateUtil.formatOfMinuteReduce(beginTime.minusMinutes(5L))), now);
|
|
|
- collector.collect(nowAdStat);
|
|
|
- lastReduceMapping.put(reduceTimeKey, nowAdStat);
|
|
|
- historyMinuteMapping.put(statDay, nowAdStat);
|
|
|
+ @Override
|
|
|
+ public void clear(TimeWindow timeWindow, TriggerContext triggerContext) throws Exception {
|
|
|
+ triggerContext.deleteEventTimeTimer(timeWindow.maxTimestamp());
|
|
|
}
|
|
|
- }).addSink(new TunnelBatchSink<>(AdStatOfMinuteDWD.class, 30000L, 365L, 6));
|
|
|
+ })
|
|
|
+ .process(new AdMinuteDWDProcess());
|
|
|
+ // .addSink(new TunnelBatchSink<>(AdStatOfMinuteDWD.class, 30000L, 365L, 6));
|
|
|
+ new KeyedBatchStream<>("adMinuteDWDStream", adMinuteDWDStream.keyBy(AdStatOfMinuteDWD::getStatDay), 4000L, 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdStatOfMinuteDWD.class));
|
|
|
|
|
|
// 小时流(直接写到小时报表的 ods)
|
|
|
DataStream<AdDataOfHourODS> adHourODSStream = adODSStream.getSideOutput(adHourStreamTag);
|
|
|
// 写入原始表
|
|
|
- adHourODSStream.addSink(new TunnelBatchSink<>(AdDataOfHourODS.class, 36000L, 64000L, 10));
|
|
|
- adHourODSStream.keyBy(AdDataOfHourODS::getAdId)
|
|
|
+ // adHourODSStream.addSink(new TunnelBatchSink<>(AdDataOfHourODS.class, 36000L, 64000L, 10));
|
|
|
+ new KeyedBatchStream<>("adHourODSStream", adHourODSStream.keyBy(AdDataOfHourODS::getStatDay), 4000L, 5 * 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdDataOfHourODS.class));
|
|
|
+
|
|
|
+ SingleOutputStreamOperator<AdStatOfHourDWD> adHourDWDStream = adHourODSStream.keyBy(AdDataOfHourODS::getAdId)
|
|
|
.countWindow(1).process(new ProcessWindowFunction<AdDataOfHourODS, AdStatOfHourDWD, Long, GlobalWindow>() {
|
|
|
private Odps odps;
|
|
|
private final Map<Long, Long> lastAdQueryTime = new ConcurrentHashMap<>();
|
|
@@ -324,48 +256,15 @@ public class AdStatJob {
|
|
|
hourDataMapping.put(newStatData.getHour(), newStatData);
|
|
|
collector.collect(newStatData);
|
|
|
}
|
|
|
- }).addSink(new TunnelBatchSink<>(AdStatOfHourDWD.class, 30000L, 365L, 6));
|
|
|
-
|
|
|
- // 获取指定广告的历史统计信息(用于统计总的消耗信息等)
|
|
|
- /*SingleOutputStreamOperator<AdStatOfHourDWD> adHourDWDStream = adHourODSStream.map(AdStatOfHourDWD::byODS)
|
|
|
- .keyBy((KeySelector<AdStatOfHourDWD, String>) adHourDWD -> adHourDWD.getStatDay() + "_" + adHourDWD.getAdId())
|
|
|
- // 开一个按天滚动的窗口
|
|
|
- .window(TumblingEventTimeWindows.of(Time.days(1L)))
|
|
|
- // 数据聚合
|
|
|
- .reduce(new RichReduceFunction<AdStatOfHourDWD>() {
|
|
|
- // 初始化 MaxCompute连接实例
|
|
|
- private Odps odps;
|
|
|
-
|
|
|
- @Override
|
|
|
- public void open(Configuration parameters) throws Exception {
|
|
|
- Map<String, String> params = getRuntimeContext()
|
|
|
- .getExecutionConfig()
|
|
|
- .getGlobalJobParameters()
|
|
|
- .toMap();
|
|
|
- Account account = new AliyunAccount(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_ID),
|
|
|
- params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_KEY));
|
|
|
- Odps odps = new Odps(account);
|
|
|
- odps.getRestClient().setRetryLogger(new MaxComputeLog());
|
|
|
- odps.setEndpoint(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_ENDPOINT));
|
|
|
- odps.setDefaultProject(params.get(ApplicationProperties.MAX_COMPUTE_ACCOUNT_PROJECT_NAME));
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public AdStatOfHourDWD reduce(AdStatOfHourDWD adStatOfHourDWD, AdStatOfHourDWD t1) {
|
|
|
- Long accountId = adStatOfHourDWD.getAccountId();
|
|
|
- Long campaignId = adStatOfHourDWD.getCampaignId();
|
|
|
- Long adId = adStatOfHourDWD.getAdId();
|
|
|
- String statDay = adStatOfHourDWD.getStatDay();
|
|
|
- // 1、从 MaxCompute取出小时数据往前的所有无重复按天统计的数据,并聚合用来算总的
|
|
|
- // 2、当前流的所有数据聚合在一起作为天数据
|
|
|
- // 3、写出到 MaxCompute
|
|
|
- return null;
|
|
|
- }
|
|
|
- });*/
|
|
|
+ });
|
|
|
+ //.addSink(new TunnelBatchSink<>(AdStatOfHourDWD.class, 30000L, 365L, 6));
|
|
|
+ new KeyedBatchStream<>("adHourDWDStream", adHourDWDStream.keyBy(AdStatOfHourDWD::getStatDay), 4000L, 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdStatOfHourDWD.class));
|
|
|
|
|
|
|
|
|
// ------------------------------------------------------- 处理广告的天数据 -----------------------------------------
|
|
|
- Properties adStreamOfDayProps = new Properties();
|
|
|
+ /*Properties adStreamOfDayProps = new Properties();
|
|
|
adStreamOfDayProps.load(AdStatJob.class.getResourceAsStream("/ad_stream_of_day.properties"));
|
|
|
KafkaSource<String> adStreamOfDaySource = buildKafkaSource(adStreamOfDayProps);
|
|
|
|
|
@@ -398,7 +297,10 @@ public class AdStatJob {
|
|
|
.adDataOfDayODS(adODS)
|
|
|
.build();
|
|
|
});
|
|
|
- adDayODSStream.map(AdStatOfDayODSDTO::getAdDataOfDayODS).addSink(new TunnelBatchSink<>(AdDataOfDayODS.class, 36000L, 6000L, 10));
|
|
|
+ // adDayODSStream.map(AdStatOfDayODSDTO::getAdDataOfDayODS).addSink(new TunnelBatchSink<>(AdDataOfDayODS.class, 36000L, 6000L, 10));
|
|
|
+ new KeyedBatchStream<>("adDayODSStream", adDayODSStream.map(AdStatOfDayODSDTO::getAdDataOfDayODS).keyBy(AdDataOfDayODS::getStatDay), 4000L, 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdDataOfDayODS.class));
|
|
|
|
|
|
// 拆分流
|
|
|
SingleOutputStreamOperator<AdStatOfDayODSDTO> adDayODSStreamSplit = adDayODSStream.process(new ProcessFunction<AdStatOfDayODSDTO, AdStatOfDayODSDTO>() {
|
|
@@ -417,7 +319,7 @@ public class AdStatJob {
|
|
|
}
|
|
|
});
|
|
|
|
|
|
- adDayODSStreamSplit.getSideOutput(adDayStreamRollDayTag)
|
|
|
+ SingleOutputStreamOperator<AdStatOfDayDWD> adDayDWDStream = adDayODSStreamSplit.getSideOutput(adDayStreamRollDayTag)
|
|
|
// 打水印,允许延迟 10天(应为允许回滚 10天),同时指定事件时间
|
|
|
.assignTimestampsAndWatermarks(WatermarkStrategy.<AdDataOfDayODS>forBoundedOutOfOrderness(Duration.ofDays(10L))
|
|
|
.withTimestampAssigner((SerializableTimestampAssigner<AdDataOfDayODS>) (adDay, l) -> DateUtil.localDateToMilli(DateUtil.parseLocalDate(adDay.getStatDay())))
|
|
@@ -479,9 +381,13 @@ public class AdStatJob {
|
|
|
historyData.put(newStatData.getStatDay(), newStatData);
|
|
|
collector.collect(newStatData);
|
|
|
}
|
|
|
- }).addSink(new TunnelBatchSink<>(AdStatOfDayDWD.class, 30000L, 365L, 6));
|
|
|
+ });
|
|
|
+ //.addSink(new TunnelBatchSink<>(AdStatOfDayDWD.class, 30000L, 365L, 6));
|
|
|
+ new KeyedBatchStream<>("adDayDWDStream", adDayDWDStream.keyBy(AdStatOfDayDWD::getStatDay), 4000L, 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdStatOfDayDWD.class));
|
|
|
|
|
|
- adDayODSStreamSplit.getSideOutput(adDayStreamRollYearTag)
|
|
|
+ SingleOutputStreamOperator<AdStatOfDayDWD> adDayDWDYearStream = adDayODSStreamSplit.getSideOutput(adDayStreamRollYearTag)
|
|
|
.keyBy((KeySelector<AdDataOfDayODS, Long>) AdDataOfDayODS::getAdId)
|
|
|
.countWindow(1).process(new ProcessWindowFunction<AdDataOfDayODS, AdStatOfDayDWD, Long, GlobalWindow>() {
|
|
|
// 上次聚合的结果
|
|
@@ -493,7 +399,11 @@ public class AdStatJob {
|
|
|
lastReduce = AdStatOfDayDWD.reduce(lastReduce, element, System.currentTimeMillis());
|
|
|
out.collect(lastReduce);
|
|
|
}
|
|
|
- }).addSink(new TunnelBatchSink<>(AdStatOfDayDWD.class, 30000L, 365L, 6));
|
|
|
+ });
|
|
|
+ //.addSink(new TunnelBatchSink<>(AdStatOfDayDWD.class, 30000L, 365L, 6));
|
|
|
+ new KeyedBatchStream<>("adDayDWDYearStream", adDayDWDYearStream.keyBy(AdStatOfDayDWD::getStatDay), 4000L, 60 * 1000L)
|
|
|
+ .toBatch()
|
|
|
+ .addSink(new TunnelBatchStreamSink<>(AdStatOfDayDWD.class));*/
|
|
|
|
|
|
env.execute();
|
|
|
}
|