Quellcode durchsuchen

增加清理逻辑

wcc vor 3 Jahren
Ursprung
Commit
d5d97b7261

+ 1 - 4
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/AdHourStatJob.java

@@ -7,7 +7,6 @@ import flink.zanxiangnet.ad.monitoring.process.*;
 import flink.zanxiangnet.ad.monitoring.sink.TunnelBatchStreamSink;
 import flink.zanxiangnet.ad.monitoring.stream.KeyedBatchStream;
 import flink.zanxiangnet.ad.monitoring.trigger.AdMinuteODSStreamTrigger;
-import flink.zanxiangnet.ad.monitoring.trigger.CostMinuteDMStreamTrigger;
 import flink.zanxiangnet.ad.monitoring.kafka.KafkaComponent;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.lang3.StringUtils;
@@ -18,10 +17,8 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
 import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.util.Collector;
 import org.apache.flink.util.OutputTag;
 
 import java.time.Duration;
@@ -114,7 +111,7 @@ public class AdHourStatJob {
 
         DataStream<AdStatOfHourDWD> adHourDWDAllStream = adMinuteDWDStream.getSideOutput(adHourFromMinuteStreamTag)
                 .keyBy(AdStatOfHourDWD::getAdId)
-                .process(new HourStreamCompletionProcess())
+                .process(new AdHourStreamCompletionProcess())
                 .union(adHourDWDStream);
         new KeyedBatchStream<>("adHourDWDStream", adHourDWDAllStream.keyBy(AdStatOfHourDWD::getStatDay), 4000L, 60 * 1000L)
                 .toBatch()

+ 5 - 1
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/PlanHourStreamJob.java

@@ -3,6 +3,7 @@ package flink.zanxiangnet.ad.monitoring;
 import flink.zanxiangnet.ad.monitoring.pojo.entity.*;
 import flink.zanxiangnet.ad.monitoring.process.PlanHourDTOStreamProcess;
 import flink.zanxiangnet.ad.monitoring.process.PlanHourDWDProcess;
+import flink.zanxiangnet.ad.monitoring.process.PlanHourStreamCompletionProcess;
 import flink.zanxiangnet.ad.monitoring.process.PlanMinuteDWDProcess;
 import flink.zanxiangnet.ad.monitoring.sink.TunnelBatchStreamSink;
 import flink.zanxiangnet.ad.monitoring.stream.KeyedBatchStream;
@@ -79,7 +80,10 @@ public class PlanHourStreamJob {
         // 小时流
         SingleOutputStreamOperator<PlanStatOfHourDWD> planHourDWDStream = adODSStream.getSideOutput(adHourStreamTag).keyBy(AdDataOfHourODS::getCampaignId)
                 .process(new PlanHourDWDProcess());
-        DataStream<PlanStatOfHourDWD> planHourDWDAllStream = planMinuteDWDStream.getSideOutput(planHourFromMinuteStreamTag).union(planHourDWDStream);
+        DataStream<PlanStatOfHourDWD> planHourDWDAllStream = planMinuteDWDStream.getSideOutput(planHourFromMinuteStreamTag)
+                .keyBy(PlanStatOfHourDWD::getCampaignId)
+                .process(new PlanHourStreamCompletionProcess())
+                .union(planHourDWDStream);
         new KeyedBatchStream<>("planHourDWDStream", planHourDWDAllStream.keyBy(PlanStatOfHourDWD::getStatDay), 4000L, 60 * 1000L)
                 .toBatch()
                 .addSink(new TunnelBatchStreamSink<>(PlanStatOfHourDWD.class))

+ 190 - 0
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/pojo/entity/PlanStatOfHourDWD.java

@@ -1795,4 +1795,194 @@ public class PlanStatOfHourDWD {
         result.setNoInterestCountHour(ObjectUtil.toLong(record.get("no_interest_count_hour")));
         return result;
     }
+
+    public static void initValue(PlanStatOfHourDWD bean) {
+        bean.setCostDeviationRateTotal(0.0);
+        bean.setCostDeviationRateDay(0.0);
+        bean.setCostDeviationRateHour(0.0);
+        bean.setCostTotal(0L);
+        bean.setCostDay(0L);
+        bean.setCostHour(0L);
+        bean.setCompensationAmountTotal(0L);
+        bean.setCompensationAmountDay(0L);
+        bean.setCompensationAmountHour(0L);
+        bean.setViewCountTotal(0L);
+        bean.setViewCountDay(0L);
+        bean.setViewCountHour(0L);
+        bean.setThousandDisplayPriceAll(0L);
+        bean.setThousandDisplayPriceDay(0L);
+        bean.setThousandDisplayPriceHour(0L);
+        bean.setAvgViewPerUserHour(0.0);
+        bean.setValidClickCountTotal(0L);
+        bean.setValidClickCountDay(0L);
+        bean.setValidClickCountHour(0L);
+        bean.setCtrAll(0.0);
+        bean.setCtrDay(0.0);
+        bean.setCtrHour(0.0);
+        bean.setCpcAll(0L);
+        bean.setCpcDay(0L);
+        bean.setCpcHour(0L);
+        bean.setValuableClickCountTotal(0L);
+        bean.setValuableClickCountDay(0L);
+        bean.setValuableClickCountHour(0L);
+        bean.setValuableClickRateAll(0.0);
+        bean.setValuableClickRateDay(0.0);
+        bean.setValuableClickRateHour(0.0);
+        bean.setValuableClickCostAll(0L);
+        bean.setValuableClickCostDay(0L);
+        bean.setValuableClickCostHour(0L);
+        bean.setConversionsCountTotal(0L);
+        bean.setConversionsCountDay(0L);
+        bean.setConversionsCountHour(0L);
+        bean.setConversionsCostAll(0L);
+        bean.setConversionsCostDay(0L);
+        bean.setConversionsCostHour(0L);
+        bean.setConversionsRateAll(0.0);
+        bean.setConversionsRateDay(0.0);
+        bean.setConversionsRateHour(0.0);
+        bean.setDeepConversionsCountTotal(0L);
+        bean.setDeepConversionsCountDay(0L);
+        bean.setDeepConversionsCountHour(0L);
+        bean.setDeepConversionsCostAll(0L);
+        bean.setDeepConversionsCostDay(0L);
+        bean.setDeepConversionsCostHour(0L);
+        bean.setDeepConversionsRateAll(0.0);
+        bean.setDeepConversionsRateDay(0.0);
+        bean.setDeepConversionsRateHour(0.0);
+        bean.setOrderCountTotal(0L);
+        bean.setOrderCountDay(0L);
+        bean.setOrderCountHour(0L);
+        bean.setFirstDayOrderCountTotal(0L);
+        bean.setFirstDayOrderCountDay(0L);
+        bean.setFirstDayOrderCountHour(0L);
+        bean.setWebOrderCostAll(0L);
+        bean.setWebOrderCostDay(0L);
+        bean.setWebOrderCostHour(0L);
+        bean.setOrderRateAll(0.0);
+        bean.setOrderRateDay(0.0);
+        bean.setOrderRateHour(0.0);
+        bean.setOrderAmountTotal(0L);
+        bean.setOrderAmountDay(0L);
+        bean.setOrderAmountHour(0L);
+        bean.setFirstDayOrderAmountTotal(0L);
+        bean.setFirstDayOrderAmountDay(0L);
+        bean.setFirstDayOrderAmountHour(0L);
+        bean.setOrderUnitPriceAll(0L);
+        bean.setOrderUnitPriceDay(0L);
+        bean.setOrderUnitPriceHour(0L);
+        bean.setOrderRoiAll(0.0);
+        bean.setOrderRoiDay(0.0);
+        bean.setOrderRoiHour(0.0);
+        bean.setSignInCountTotal(0L);
+        bean.setSignInCountDay(0L);
+        bean.setSignInCountHour(0L);
+        bean.setScanFollowCountTotal(0L);
+        bean.setScanFollowCountDay(0L);
+        bean.setScanFollowCountHour(0L);
+        bean.setWechatAppRegisterUvTotal(0L);
+        bean.setWechatAppRegisterUvDay(0L);
+        bean.setWechatAppRegisterUvHour(0L);
+        bean.setWechatMinigameRegisterCostAll(0L);
+        bean.setWechatMinigameRegisterCostDay(0L);
+        bean.setWechatMinigameRegisterCostHour(0L);
+        bean.setWechatMinigameRegisterRateAll(0.0);
+        bean.setWechatMinigameRegisterRateDay(0.0);
+        bean.setWechatMinigameRegisterRateHour(0.0);
+        bean.setWechatMinigameArpuAll(0.0);
+        bean.setWechatMinigameArpuDay(0.0);
+        bean.setWechatMinigameArpuHour(0.0);
+        bean.setWechatMinigameRetentionCountTotal(0L);
+        bean.setWechatMinigameRetentionCountDay(0L);
+        bean.setWechatMinigameRetentionCountHour(0L);
+        bean.setWechatMinigameCheckoutCountTotal(0L);
+        bean.setWechatMinigameCheckoutCountDay(0L);
+        bean.setWechatMinigameCheckoutCountHour(0L);
+        bean.setWechatMinigameCheckoutAmountTotal(0L);
+        bean.setWechatMinigameCheckoutAmountDay(0L);
+        bean.setWechatMinigameCheckoutAmountHour(0L);
+        bean.setOfficialAccountFollowCountTotal(0L);
+        bean.setOfficialAccountFollowCountDay(0L);
+        bean.setOfficialAccountFollowCountHour(0L);
+        bean.setOfficialAccountFollowRateAll(0.0);
+        bean.setOfficialAccountFollowRateDay(0.0);
+        bean.setOfficialAccountFollowRateHour(0.0);
+        bean.setOfficialAccountRegisterUserCountTotal(0L);
+        bean.setOfficialAccountRegisterUserCountDay(0L);
+        bean.setOfficialAccountRegisterUserCountHour(0L);
+        bean.setOfficialAccountRegisterRateAll(0.0);
+        bean.setOfficialAccountRegisterRateDay(0.0);
+        bean.setOfficialAccountRegisterRateHour(0.0);
+        bean.setOfficialAccountRegisterCostAll(0L);
+        bean.setOfficialAccountRegisterCostDay(0L);
+        bean.setOfficialAccountRegisterCostHour(0L);
+        bean.setOfficialAccountRegisterAmountTotal(0L);
+        bean.setOfficialAccountRegisterAmountDay(0L);
+        bean.setOfficialAccountRegisterAmountHour(0L);
+        bean.setOfficialAccountRegisterRoiAll(0L);
+        bean.setOfficialAccountRegisterRoiDay(0L);
+        bean.setOfficialAccountRegisterRoiHour(0L);
+        bean.setOfficialAccountApplyCountTotal(0L);
+        bean.setOfficialAccountApplyCountDay(0L);
+        bean.setOfficialAccountApplyCountHour(0L);
+        bean.setOfficialAccountApplyUserCountTotal(0L);
+        bean.setOfficialAccountApplyUserCountDay(0L);
+        bean.setOfficialAccountApplyUserCountHour(0L);
+        bean.setOfficialAccountApplyRateAll(0.0);
+        bean.setOfficialAccountApplyRateDay(0.0);
+        bean.setOfficialAccountApplyRateHour(0.0);
+        bean.setOfficialAccountApplyCostAll(0L);
+        bean.setOfficialAccountApplyCostDay(0L);
+        bean.setOfficialAccountApplyCostHour(0L);
+        bean.setOfficialAccountApplyAmountTotal(0L);
+        bean.setOfficialAccountApplyAmountDay(0L);
+        bean.setOfficialAccountApplyAmountHour(0L);
+        bean.setOfficialAccountApplyRoiAll(0L);
+        bean.setOfficialAccountApplyRoiDay(0L);
+        bean.setOfficialAccountApplyRoiHour(0L);
+        bean.setOfficialAccountOrderCountTotal(0L);
+        bean.setOfficialAccountOrderCountDay(0L);
+        bean.setOfficialAccountOrderCountHour(0L);
+        bean.setOfficialAccountFirstDayOrderCountTotal(0L);
+        bean.setOfficialAccountFirstDayOrderCountDay(0L);
+        bean.setOfficialAccountFirstDayOrderCountHour(0L);
+        bean.setOfficialAccountOrderUserCountTotal(0L);
+        bean.setOfficialAccountOrderUserCountDay(0L);
+        bean.setOfficialAccountOrderUserCountHour(0L);
+        bean.setOfficialAccountOrderRateAll(0.0);
+        bean.setOfficialAccountOrderRateDay(0.0);
+        bean.setOfficialAccountOrderRateHour(0.0);
+        bean.setOfficialAccountOrderCostAll(0L);
+        bean.setOfficialAccountOrderCostDay(0L);
+        bean.setOfficialAccountOrderCostHour(0L);
+        bean.setOfficialAccountOrderAmountTotal(0L);
+        bean.setOfficialAccountOrderAmountDay(0L);
+        bean.setOfficialAccountOrderAmountHour(0L);
+        bean.setOfficialAccountFirstDayOrderAmountTotal(0L);
+        bean.setOfficialAccountFirstDayOrderAmountDay(0L);
+        bean.setOfficialAccountFirstDayOrderAmountHour(0L);
+        bean.setOfficialAccountOrderRoiAll(0L);
+        bean.setOfficialAccountOrderRoiDay(0L);
+        bean.setOfficialAccountOrderRoiHour(0L);
+        bean.setOfficialAccountConsultCountTotal(0L);
+        bean.setOfficialAccountConsultCountDay(0L);
+        bean.setOfficialAccountConsultCountHour(0L);
+        bean.setOfficialAccountReaderCountTotal(0L);
+        bean.setOfficialAccountReaderCountDay(0L);
+        bean.setOfficialAccountReaderCountHour(0L);
+        bean.setOfficialAccountCreditApplyUserCountTotal(0L);
+        bean.setOfficialAccountCreditApplyUserCountDay(0L);
+        bean.setOfficialAccountCreditApplyUserCountHour(0L);
+        bean.setOfficialAccountCreditUserCountTotal(0L);
+        bean.setOfficialAccountCreditUserCountDay(0L);
+        bean.setOfficialAccountCreditUserCountHour(0L);
+        bean.setForwardCountTotal(0L);
+        bean.setForwardCountDay(0L);
+        bean.setForwardCountHour(0L);
+        bean.setForwardUserCountTotal(0L);
+        bean.setForwardUserCountDay(0L);
+        bean.setForwardUserCountHour(0L);
+        bean.setNoInterestCountTotal(0L);
+        bean.setNoInterestCountDay(0L);
+        bean.setNoInterestCountHour(0L);
+    }
 }

+ 1 - 1
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/process/HourStreamCompletionProcess.java → flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/process/AdHourStreamCompletionProcess.java

@@ -17,7 +17,7 @@ import java.util.Date;
 /**
  * 小时流数据补全,把中间没消耗的时间段填充 0
  */
-public class HourStreamCompletionProcess extends KeyedProcessFunction<Long, AdStatOfHourDWD, AdStatOfHourDWD> {
+public class AdHourStreamCompletionProcess extends KeyedProcessFunction<Long, AdStatOfHourDWD, AdStatOfHourDWD> {
 
     private ValueState<AdStatOfHourDWD> lastReduceState;
 

+ 197 - 0
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/process/PlanHourStreamCompletionProcess.java

@@ -0,0 +1,197 @@
+package flink.zanxiangnet.ad.monitoring.process;
+
+import flink.zanxiangnet.ad.monitoring.pojo.entity.AdStatOfHourDWD;
+import flink.zanxiangnet.ad.monitoring.pojo.entity.PlanStatOfHourDWD;
+import flink.zanxiangnet.ad.monitoring.util.DateUtil;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.springframework.beans.BeanUtils;
+
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Date;
+
+/**
+ * 小时流数据补全,把中间没消耗的时间段填充 0
+ */
+public class PlanHourStreamCompletionProcess extends KeyedProcessFunction<Long, PlanStatOfHourDWD, PlanStatOfHourDWD> {
+
+    private ValueState<PlanStatOfHourDWD> lastReduceState;
+
+    @Override
+    public void open(Configuration conf) {
+        lastReduceState = getRuntimeContext().getState(new ValueStateDescriptor<>("lastReduceState", Types.POJO(PlanStatOfHourDWD.class)));
+    }
+
+    @Override
+    public void processElement(PlanStatOfHourDWD planStatOfHourDWD, KeyedProcessFunction<Long, PlanStatOfHourDWD, PlanStatOfHourDWD>.Context context,
+                               Collector<PlanStatOfHourDWD> collector) throws Exception {
+        PlanStatOfHourDWD lastReduce = lastReduceState.value();
+        if (lastReduce == null) {
+            lastReduceState.update(planStatOfHourDWD);
+            collector.collect(planStatOfHourDWD);
+            return;
+        }
+        LocalDateTime statDateTime = LocalDateTime.of(DateUtil.parseLocalDate(planStatOfHourDWD.getStatDay()), LocalTime.of(planStatOfHourDWD.getHour(), 0, 0));
+        LocalDateTime lastStatDateTime = LocalDateTime.of(DateUtil.parseLocalDate(lastReduce.getStatDay()), LocalTime.of(lastReduce.getHour(), 0, 0));
+        if (lastStatDateTime.compareTo(statDateTime) >= 0) {
+            return;
+        }
+        long hours = DateUtil.intervalOfHour(lastStatDateTime, statDateTime);
+        if (hours > 1) {
+            // 中间有没数据的时间段,需要进行数据填充
+            for (int i = 1; i < hours; i++) {
+                // 要填充的时间
+                LocalDateTime completionTime = lastStatDateTime.plusHours(i);
+                collector.collect(completion(completionTime, lastReduce));
+            }
+        }
+        collector.collect(planStatOfHourDWD);
+        lastReduceState.update(planStatOfHourDWD);
+    }
+
+    private PlanStatOfHourDWD completion(LocalDateTime completionTime, PlanStatOfHourDWD lastReduceData) {
+        String statDay = DateUtil.formatLocalDate(completionTime.toLocalDate());
+        int hour = completionTime.getHour();
+        PlanStatOfHourDWD result = new PlanStatOfHourDWD();
+        BeanUtils.copyProperties(lastReduceData, result);
+        result.setCreateTime(new Date());
+        result.setStatDay(statDay);
+        result.setHour(hour);
+        PlanStatOfHourDWD.initValue(result);
+
+        result.setCostDeviationRateTotal(lastReduceData.getCostDeviationRateTotal());
+        result.setCostTotal(lastReduceData.getCostTotal());
+        result.setCompensationAmountTotal(lastReduceData.getCompensationAmountTotal());
+        result.setViewCountTotal(lastReduceData.getViewCountTotal());
+        result.setThousandDisplayPriceAll(lastReduceData.getThousandDisplayPriceAll());
+        result.setValidClickCountTotal(lastReduceData.getValidClickCountTotal());
+        result.setCtrAll(lastReduceData.getCtrAll());
+        result.setCpcAll(lastReduceData.getCpcAll());
+        result.setValuableClickCountTotal(lastReduceData.getValuableClickCountTotal());
+        result.setValuableClickRateAll(lastReduceData.getValuableClickRateAll());
+        result.setValuableClickCostAll(lastReduceData.getValuableClickCostAll());
+        result.setConversionsCountTotal(lastReduceData.getConversionsCountTotal());
+        result.setConversionsCostAll(lastReduceData.getConversionsCostAll());
+        result.setConversionsRateAll(lastReduceData.getConversionsRateAll());
+        result.setDeepConversionsCountTotal(lastReduceData.getDeepConversionsCountTotal());
+        result.setDeepConversionsCostAll(lastReduceData.getDeepConversionsCostAll());
+        result.setDeepConversionsRateAll(lastReduceData.getDeepConversionsRateAll());
+        result.setOrderCountTotal(lastReduceData.getOrderCountTotal());
+        result.setFirstDayOrderCountTotal(lastReduceData.getFirstDayOrderCountTotal());
+        result.setWebOrderCostAll(lastReduceData.getWebOrderCostAll());
+        result.setOrderRateAll(lastReduceData.getOrderRateAll());
+        result.setOrderAmountTotal(lastReduceData.getOrderAmountTotal());
+        result.setFirstDayOrderAmountTotal(lastReduceData.getFirstDayOrderAmountTotal());
+        result.setOrderUnitPriceAll(lastReduceData.getOrderUnitPriceAll());
+        result.setOrderRoiAll(lastReduceData.getOrderRoiAll());
+        result.setSignInCountTotal(lastReduceData.getSignInCountTotal());
+        result.setScanFollowCountTotal(lastReduceData.getScanFollowCountTotal());
+        result.setWechatAppRegisterUvTotal(lastReduceData.getWechatAppRegisterUvTotal());
+        result.setWechatMinigameRegisterCostAll(lastReduceData.getWechatMinigameRegisterCostAll());
+        result.setWechatMinigameRegisterRateAll(lastReduceData.getWechatMinigameRegisterRateAll());
+        result.setWechatMinigameArpuAll(lastReduceData.getWechatMinigameArpuAll());
+        result.setWechatMinigameRetentionCountTotal(lastReduceData.getWechatMinigameRetentionCountTotal());
+        result.setWechatMinigameCheckoutCountTotal(lastReduceData.getWechatMinigameCheckoutCountTotal());
+        result.setWechatMinigameCheckoutAmountTotal(lastReduceData.getWechatMinigameCheckoutAmountTotal());
+        result.setOfficialAccountFollowCountTotal(lastReduceData.getOfficialAccountFollowCountTotal());
+        result.setOfficialAccountFollowRateAll(lastReduceData.getOfficialAccountFollowRateAll());
+        result.setOfficialAccountRegisterUserCountTotal(lastReduceData.getOfficialAccountRegisterUserCountTotal());
+        result.setOfficialAccountRegisterRateAll(lastReduceData.getOfficialAccountRegisterRateAll());
+        result.setOfficialAccountRegisterCostAll(lastReduceData.getOfficialAccountRegisterCostAll());
+        result.setOfficialAccountRegisterAmountTotal(lastReduceData.getOfficialAccountRegisterAmountTotal());
+        result.setOfficialAccountRegisterRoiAll(lastReduceData.getOfficialAccountRegisterRoiAll());
+        result.setOfficialAccountApplyCountTotal(lastReduceData.getOfficialAccountApplyCountTotal());
+        result.setOfficialAccountApplyUserCountTotal(lastReduceData.getOfficialAccountApplyUserCountTotal());
+        result.setOfficialAccountApplyRateAll(lastReduceData.getOfficialAccountApplyRateAll());
+        result.setOfficialAccountApplyCostAll(lastReduceData.getOfficialAccountApplyCostAll());
+        result.setOfficialAccountApplyAmountTotal(lastReduceData.getOfficialAccountApplyAmountTotal());
+        result.setOfficialAccountApplyRoiAll(lastReduceData.getOfficialAccountApplyRoiAll());
+        result.setOfficialAccountOrderCountTotal(lastReduceData.getOfficialAccountOrderCountTotal());
+        result.setOfficialAccountFirstDayOrderCountTotal(lastReduceData.getOfficialAccountFirstDayOrderCountTotal());
+        result.setOfficialAccountOrderUserCountTotal(lastReduceData.getOfficialAccountOrderUserCountTotal());
+        result.setOfficialAccountOrderRateAll(lastReduceData.getOfficialAccountOrderRateAll());
+        result.setOfficialAccountOrderCostAll(lastReduceData.getOfficialAccountOrderCostAll());
+        result.setOfficialAccountOrderAmountTotal(lastReduceData.getOfficialAccountOrderAmountTotal());
+        result.setOfficialAccountFirstDayOrderAmountTotal(lastReduceData.getOfficialAccountFirstDayOrderAmountTotal());
+        result.setOfficialAccountOrderRoiAll(lastReduceData.getOfficialAccountOrderRoiAll());
+        result.setOfficialAccountConsultCountTotal(lastReduceData.getOfficialAccountConsultCountTotal());
+        result.setOfficialAccountReaderCountTotal(lastReduceData.getOfficialAccountReaderCountTotal());
+        result.setOfficialAccountCreditApplyUserCountTotal(lastReduceData.getOfficialAccountCreditApplyUserCountTotal());
+        result.setOfficialAccountCreditUserCountTotal(lastReduceData.getOfficialAccountCreditUserCountTotal());
+        result.setForwardCountTotal(lastReduceData.getForwardCountTotal());
+        result.setForwardUserCountTotal(lastReduceData.getForwardUserCountTotal());
+        result.setNoInterestCountTotal(lastReduceData.getNoInterestCountTotal());
+        result.setNoInterestCountHour(lastReduceData.getNoInterestCountHour());
+
+        if (statDay.equals(lastReduceData.getStatDay())) {
+            result.setCostDeviationRateDay(lastReduceData.getCostDeviationRateDay());
+            result.setCostDay(lastReduceData.getCostDay());
+            result.setCompensationAmountDay(lastReduceData.getCompensationAmountDay());
+            result.setViewCountDay(lastReduceData.getViewCountDay());
+            result.setThousandDisplayPriceDay(lastReduceData.getThousandDisplayPriceDay());
+            result.setValidClickCountDay(lastReduceData.getValidClickCountDay());
+            result.setCtrDay(lastReduceData.getCtrDay());
+            result.setCpcDay(lastReduceData.getCpcDay());
+            result.setValuableClickCountDay(lastReduceData.getValuableClickCountDay());
+            result.setValuableClickRateDay(lastReduceData.getValuableClickRateDay());
+            result.setValuableClickCostDay(lastReduceData.getValuableClickCostDay());
+            result.setConversionsCountDay(lastReduceData.getConversionsCountDay());
+            result.setConversionsCostDay(lastReduceData.getConversionsCostDay());
+            result.setConversionsRateDay(lastReduceData.getConversionsRateDay());
+            result.setDeepConversionsCountDay(lastReduceData.getDeepConversionsCountDay());
+            result.setDeepConversionsCostDay(lastReduceData.getDeepConversionsCostDay());
+            result.setDeepConversionsRateDay(lastReduceData.getDeepConversionsRateDay());
+            result.setOrderCountDay(lastReduceData.getOrderCountDay());
+            result.setFirstDayOrderCountDay(lastReduceData.getFirstDayOrderCountDay());
+            result.setWebOrderCostDay(lastReduceData.getWebOrderCostDay());
+            result.setOrderRateDay(lastReduceData.getOrderRateDay());
+            result.setOrderAmountDay(lastReduceData.getOrderAmountDay());
+            result.setFirstDayOrderAmountDay(lastReduceData.getFirstDayOrderAmountDay());
+            result.setOrderUnitPriceDay(lastReduceData.getOrderUnitPriceDay());
+            result.setOrderRoiDay(lastReduceData.getOrderRoiDay());
+            result.setSignInCountDay(lastReduceData.getSignInCountDay());
+            result.setScanFollowCountDay(lastReduceData.getScanFollowCountDay());
+            result.setWechatAppRegisterUvDay(lastReduceData.getWechatAppRegisterUvDay());
+            result.setWechatMinigameRegisterCostDay(lastReduceData.getWechatMinigameRegisterCostDay());
+            result.setWechatMinigameRegisterRateDay(lastReduceData.getWechatMinigameRegisterRateDay());
+            result.setWechatMinigameArpuDay(lastReduceData.getWechatMinigameArpuDay());
+            result.setWechatMinigameRetentionCountDay(lastReduceData.getWechatMinigameRetentionCountDay());
+            result.setWechatMinigameCheckoutCountDay(lastReduceData.getWechatMinigameCheckoutCountDay());
+            result.setWechatMinigameCheckoutAmountDay(lastReduceData.getWechatMinigameCheckoutAmountDay());
+            result.setOfficialAccountFollowCountDay(lastReduceData.getOfficialAccountFollowCountDay());
+            result.setOfficialAccountFollowRateDay(lastReduceData.getOfficialAccountFollowRateDay());
+            result.setOfficialAccountRegisterUserCountDay(lastReduceData.getOfficialAccountRegisterUserCountDay());
+            result.setOfficialAccountRegisterRateDay(lastReduceData.getOfficialAccountRegisterRateDay());
+            result.setOfficialAccountRegisterCostDay(lastReduceData.getOfficialAccountRegisterCostDay());
+            result.setOfficialAccountRegisterAmountDay(lastReduceData.getOfficialAccountRegisterAmountDay());
+            result.setOfficialAccountRegisterRoiDay(lastReduceData.getOfficialAccountRegisterRoiDay());
+            result.setOfficialAccountApplyCountDay(lastReduceData.getOfficialAccountApplyCountDay());
+            result.setOfficialAccountApplyUserCountDay(lastReduceData.getOfficialAccountApplyUserCountDay());
+            result.setOfficialAccountApplyRateDay(lastReduceData.getOfficialAccountApplyRateDay());
+            result.setOfficialAccountApplyCostDay(lastReduceData.getOfficialAccountApplyCostDay());
+            result.setOfficialAccountApplyAmountDay(lastReduceData.getOfficialAccountApplyAmountDay());
+            result.setOfficialAccountApplyRoiDay(lastReduceData.getOfficialAccountApplyRoiDay());
+            result.setOfficialAccountOrderCountDay(lastReduceData.getOfficialAccountOrderCountDay());
+            result.setOfficialAccountFirstDayOrderCountDay(lastReduceData.getOfficialAccountFirstDayOrderCountDay());
+            result.setOfficialAccountOrderUserCountDay(lastReduceData.getOfficialAccountOrderUserCountDay());
+            result.setOfficialAccountOrderRateDay(lastReduceData.getOfficialAccountOrderRateDay());
+            result.setOfficialAccountOrderCostDay(lastReduceData.getOfficialAccountOrderCostDay());
+            result.setOfficialAccountOrderAmountDay(lastReduceData.getOfficialAccountOrderAmountDay());
+            result.setOfficialAccountFirstDayOrderAmountDay(lastReduceData.getOfficialAccountFirstDayOrderAmountDay());
+            result.setOfficialAccountOrderRoiDay(lastReduceData.getOfficialAccountOrderRoiDay());
+            result.setOfficialAccountConsultCountDay(lastReduceData.getOfficialAccountConsultCountDay());
+            result.setOfficialAccountReaderCountDay(lastReduceData.getOfficialAccountReaderCountDay());
+            result.setOfficialAccountCreditApplyUserCountDay(lastReduceData.getOfficialAccountCreditApplyUserCountDay());
+            result.setOfficialAccountCreditUserCountDay(lastReduceData.getOfficialAccountCreditUserCountDay());
+            result.setForwardCountDay(lastReduceData.getForwardCountDay());
+            result.setForwardUserCountDay(lastReduceData.getForwardUserCountDay());
+            result.setNoInterestCountDay(lastReduceData.getNoInterestCountDay());
+        }
+        return result;
+    }
+}