wcc 3 سال پیش
والد
کامیت
626942bfdb

+ 20 - 13
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/KafkaDemoJob.java

@@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.connector.sink.Sink;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -34,8 +35,10 @@ import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 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.windows.GlobalWindow;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.OutputTag;
 import org.apache.kafka.clients.CommonClientConfigs;
@@ -50,6 +53,7 @@ import java.time.Duration;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -187,7 +191,7 @@ public class KafkaDemoJob {
         final OutputTag<AdDataOfDayODS> adDayStreamRollDayTag = new OutputTag<AdDataOfDayODS>("adDayStreamRollDayTag") {
         };
         // 广告日数据。往前回滚 1年
-        final OutputTag<AdStatOfDayDWDDTO> adDayStreamRollYearTag = new OutputTag<AdStatOfDayDWDDTO>("adDayStreamRollYearTag") {
+        final OutputTag<AdDataOfDayODS> adDayStreamRollYearTag = new OutputTag<AdDataOfDayODS>("adDayStreamRollYearTag") {
         };
         SingleOutputStreamOperator<AdStatOfDayODSDTO> adDayOdsStream = adStreamOfDayIn.filter(StringUtils::isNotBlank)
                 .map(str -> {
@@ -221,11 +225,7 @@ public class KafkaDemoJob {
                 AdDataOfDayODS adOds = adDataDTO.getAdDataOfDayODS();
                 if (DateUtil.intervalOfDays(startDate, endDate) > 31L) {
                     // 拉取时间间隔超过 1个月,账号回滚 365天的数据
-                    context.output(adDayStreamRollYearTag, AdStatOfDayDWDDTO.builder()
-                            .startDate(startDate)
-                            .endDate(endDate)
-                            .adStatOfDayDWD(AdStatOfDayDWD.byOds(adOds))
-                            .build());
+                    context.output(adDayStreamRollYearTag, adOds);
                 } else {
                     // 每日往前回滚 10天的数据
                     context.output(adDayStreamRollDayTag, adOds);
@@ -239,13 +239,20 @@ public class KafkaDemoJob {
                         .withTimestampAssigner((SerializableTimestampAssigner<AdDataOfDayODS>) (adDay, l) -> DateUtil.localDateToMilli(DateUtil.parseLocalDate(adDay.getStatDay())))
                 );
 
-        /*SingleOutputStreamOperator<AdStatOfDayDWDDTO> adDayOdsOfYearStream = adDayOdsStreamSplit.getSideOutput(adDayStreamRollYearTag)
-                // 打水印,允许延迟 365天(应为允许回滚 365天),同时指定事件时间
-                .assignTimestampsAndWatermarks(WatermarkStrategy.<AdStatOfDayDWDDTO>forBoundedOutOfOrderness(Duration.ofDays(366L))
-                        .withTimestampAssigner((SerializableTimestampAssigner<AdStatOfDayDWDDTO>) (dto, l) -> DateUtil.localDateToMilli(DateUtil.parseLocalDate(dto.getAdStatOfDayDWD().getStatDay()))))
-                .keyBy((KeySelector<AdStatOfDayDWD, String>) adStatOfDayDWD -> adStatOfDayDWD.getStatDay() + "_" + adStatOfDayDWD.getAdId())
-                // 开一个以天为单位的窗口
-                .window(TumblingEventTimeWindows.of(Time.days(1L), Time.hours(-8)));*/
+        adDayOdsStreamSplit.getSideOutput(adDayStreamRollYearTag)
+                .keyBy((KeySelector<AdDataOfDayODS, Long>) AdDataOfDayODS::getAdId)
+                .countWindow(1).process(new ProcessWindowFunction<AdDataOfDayODS, AdStatOfDayDWD, Long, GlobalWindow>() {
+                    private Long reduceCount = 0L;
+                    // 上次聚合的结果
+                    private AdStatOfDayDWD lastReduce;
+
+                    @Override
+                    public void process(Long elementsCount, ProcessWindowFunction<AdDataOfDayODS, AdStatOfDayDWD, Long, GlobalWindow>.Context context, Iterable<AdDataOfDayODS> elements, Collector<AdStatOfDayDWD> out) {
+                        AdDataOfDayODS element = elements.iterator().next();
+                        lastReduce = AdStatOfDayDWD.reduce(reduceCount++, lastReduce, element);
+                        out.collect(lastReduce);
+                    }
+                }).addSink(new TunnelBatchSink<>(AdStatOfDayDWD.class, 30000L, 365L, 6));
 
         /*adOdsStream.join(oldAdStream)
                 .where((KeySelector<AdDataOfMinuteODS, Long>) AdDataOfMinuteODS::getAdId)

+ 50 - 3
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/Test.java

@@ -2,6 +2,7 @@ package flink.zanxiangnet.ad.monitoring;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import flink.zanxiangnet.ad.monitoring.util.DateUtil;
+import flink.zanxiangnet.ad.monitoring.util.JsonUtil;
 import lombok.AllArgsConstructor;
 import lombok.Builder;
 import lombok.Data;
@@ -9,22 +10,32 @@ import lombok.NoArgsConstructor;
 import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
 import org.apache.flink.api.common.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.functions.RichAggregateFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.configuration.Configuration;
 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.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 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.windows.GlobalWindow;
+import org.apache.flink.util.Collector;
 
 import java.time.Duration;
+import java.time.LocalDateTime;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 
 public class Test {
     public static void main(String[] args) throws Exception {
@@ -35,7 +46,7 @@ public class Test {
         SingleOutputStreamOperator<Pojo> pojoStream = source.assignTimestampsAndWatermarks(WatermarkStrategy.<Pojo>forBoundedOutOfOrderness(Duration.ofHours(2))
                 .withTimestampAssigner((SerializableTimestampAssigner<Pojo>) (pojo, l) -> pojo.getCreateTime())
         );
-        pojoStream.keyBy(Pojo::getUserId)
+        /*pojoStream.keyBy(Pojo::getUserId)
                 .window(TumblingEventTimeWindows.of(Time.days(1L), Time.hours(-8)))
                 .aggregate(new AggregateFunction<Pojo, Tuple5<Integer, Long, Long, Integer, List<Long>>, String>() {
 
@@ -61,7 +72,43 @@ public class Test {
                         temp.addAll(acc.f4);
                         return new Tuple5<>(tuple5.f0, Math.min(acc.f1, tuple5.f1), Math.max(acc.f2, tuple5.f2), tuple5.f3 + acc.f3, temp);
                     }
+                }).print();*/
+        pojoStream.keyBy(Pojo::getUserId).countWindow(1)
+                .process(new ProcessWindowFunction<Pojo, String, Integer, GlobalWindow>() {
+                    private Tuple3<LocalDateTime, LocalDateTime, List<Pojo>> oldData = null;
+                    @Override
+                    public void process(Integer integer, ProcessWindowFunction<Pojo, String, Integer, GlobalWindow>.Context context, Iterable<Pojo> elements, Collector<String> out) throws Exception {
+                        System.out.println(integer);
+                        Pojo pojo = null;
+                        for(Pojo obj : elements) {
+                            pojo = obj;
+                        }
+                        LocalDateTime createTime = DateUtil.milliToLocalDateTime(pojo.getCreateTime());
+                        if (oldData == null || createTime.getDayOfYear() != oldData.f1.getDayOfYear()) {
+                            oldData = new Tuple3<>(createTime, createTime, new ArrayList<>(200));
+                        }
+                        oldData.f2.add(pojo);
+                        oldData.setField(createTime, 1);
+                        out.collect(JsonUtil.toString(oldData.f2.stream().map(Pojo::getIndex).collect(Collectors.toList())));
+                    }
                 }).print();
+/*        .apply(new WindowFunction<Pojo, String, Integer, GlobalWindow>() {
+            private Tuple3<LocalDateTime, LocalDateTime, List<Pojo>> oldData = null;
+            @Override
+            public void apply(Integer integer, GlobalWindow window, Iterable<Pojo> input, Collector<String> out) throws Exception {
+                Pojo pojo = null;
+                for(Pojo obj : input) {
+                    pojo = obj;
+                }
+                LocalDateTime createTime = DateUtil.milliToLocalDateTime(pojo.getCreateTime());
+                if (oldData == null || createTime.getDayOfYear() != oldData.f1.getDayOfYear()) {
+                    oldData = new Tuple3<>(createTime, createTime, new ArrayList<>(200));
+                }
+                oldData.f2.add(pojo);
+                oldData.setField(createTime, 1);
+                out.collect(JsonUtil.toString(oldData.f2.stream().map(Pojo::getIndex).collect(Collectors.toList())));
+            }
+        })*/
 
         env.execute();
     }
@@ -127,7 +174,7 @@ public class Test {
                         System.out.println("1发送: " + pojo);
                         sourceContext.collect(pojo);
                     }
-                    long user2 = index2.incrementAndGet();
+                    /*long user2 = index2.incrementAndGet();
                     Pojo pojo2 = Pojo.builder()
                             .userId(2)
                             .index(user2)
@@ -147,7 +194,7 @@ public class Test {
                     } else {
                         System.out.println("2发送: " + pojo2);
                         sourceContext.collect(pojo2);
-                    }
+                    }*/
                     Thread.sleep(1000);
                 } catch (Exception e) {
                     e.printStackTrace();

+ 348 - 0
flink-ad-monitoring/src/main/java/flink/zanxiangnet/ad/monitoring/pojo/entity/AdStatOfDayDWD.java

@@ -4,8 +4,10 @@ import com.google.gson.annotations.SerializedName;
 import flink.zanxiangnet.ad.monitoring.maxcompute.bean.annotation.MaxComputeColumn;
 import flink.zanxiangnet.ad.monitoring.maxcompute.bean.annotation.MaxComputeTable;
 import lombok.Data;
+import org.springframework.beans.BeanUtils;
 
 import java.io.Serializable;
+import java.math.BigDecimal;
 import java.util.Date;
 
 @Data
@@ -1030,4 +1032,350 @@ public class AdStatOfDayDWD implements Serializable {
         adStatOfDay.setNoInterestCountDay(dayOds.getNoInterestCount());
         return adStatOfDay;
     }
+
+    /**
+     * 聚合
+     *
+     * @param reduceCount 之前聚合的次数
+     * @param value1      之前聚合的值(可能为 null)
+     * @param value2      新值
+     * @return
+     */
+    public static AdStatOfDayDWD reduce(Long reduceCount, AdStatOfDayDWD value1, AdDataOfDayODS value2) {
+        AdStatOfDayDWD result = new AdStatOfDayDWD();
+        BeanUtils.copyProperties(value2, result);
+        if (value1 == null) {
+            result.setCostDeviationRateTotal(value2.getCostDeviationRate());
+            result.setCostDeviationRateDay(value2.getCostDeviationRate());
+            result.setCostTotal(value2.getCost());
+            result.setCostDay(value2.getCost());
+            result.setCompensationAmountTotal(value2.getCompensationAmount());
+            result.setCompensationAmountDay(value2.getCompensationAmount());
+            result.setViewCountTotal(value2.getViewCount());
+            result.setViewCountDay(value2.getViewCount());
+            result.setThousandDisplayPriceAll(value2.getThousandDisplayPrice());
+            result.setThousandDisplayPriceDay(value2.getThousandDisplayPrice());
+            result.setViewUserCountTotal(value2.getViewUserCount());
+            result.setViewUserCountDay(value2.getViewUserCount());
+            result.setAvgViewPerUserAll(value2.getAvgViewPerUser());
+            result.setAvgViewPerUserDay(value2.getAvgViewPerUser());
+            result.setValidClickCountTotal(value2.getValidClickCount());
+            result.setValidClickCountDay(value2.getValidClickCount());
+            result.setClickUserCountTotal(value2.getClickUserCount());
+            result.setClickUserCountDay(value2.getClickUserCount());
+            result.setCtrAll(value2.getCtr());
+            result.setCtrDay(value2.getCtr());
+            result.setCpcAll(value2.getCpc());
+            result.setCpcDay(value2.getCpc());
+            result.setValuableClickCountTotal(value2.getValuableClickCount());
+            result.setValuableClickCountDay(value2.getValuableClickCount());
+            result.setValuableClickRateAll(value2.getValuableClickRate());
+            result.setValuableClickRateDay(value2.getValuableClickRate());
+            result.setValuableClickCostAll(value2.getValuableClickCost());
+            result.setValuableClickCostDay(value2.getValuableClickCost());
+            result.setConversionsCountTotal(value2.getConversionsCount());
+            result.setConversionsCountDay(value2.getConversionsCount());
+            result.setConversionsCostAll(value2.getConversionsCost());
+            result.setConversionsCostDay(value2.getConversionsCost());
+            result.setConversionsRateAll(value2.getConversionsRate());
+            result.setConversionsRateDay(value2.getConversionsRate());
+            result.setDeepConversionsCountTotal(value2.getDeepConversionsCount());
+            result.setDeepConversionsCountDay(value2.getDeepConversionsCount());
+            result.setDeepConversionsCostTotal(value2.getDeepConversionsCost());
+            result.setDeepConversionsCostDay(value2.getDeepConversionsCost());
+            result.setDeepConversionsRateTotal(value2.getDeepConversionsRate());
+            result.setDeepConversionsRateDay(value2.getDeepConversionsRate());
+            result.setKeyPageUvTotal(value2.getKeyPageUv());
+            result.setKeyPageUvDay(value2.getKeyPageUv());
+            result.setOrderCountTotal(value2.getOrderCount());
+            result.setOrderCountDay(value2.getOrderCount());
+            result.setFirstDayOrderCountTotal(value2.getFirstDayOrderCount());
+            result.setFirstDayOrderCountDay(value2.getFirstDayOrderCount());
+            result.setWebOrderCostAll(value2.getWebOrderCost());
+            result.setWebOrderCostDay(value2.getWebOrderCost());
+            result.setOrderRateAll(value2.getOrderRate());
+            result.setOrderRateDay(value2.getOrderRate());
+            result.setOrderAmountTotal(value2.getOrderAmount());
+            result.setOrderAmountDay(value2.getOrderAmount());
+            result.setFirstDayOrderAmountTotal(value2.getFirstDayOrderAmount());
+            result.setFirstDayOrderAmountDay(value2.getFirstDayOrderAmount());
+            result.setOrderUnitPriceAll(value2.getOrderUnitPrice());
+            result.setOrderUnitPriceDay(value2.getOrderUnitPrice());
+            result.setOrderRoiAll(value2.getOrderRoi());
+            result.setOrderRoiDay(value2.getOrderRoi());
+            result.setSignInCountTotal(value2.getSignInCount());
+            result.setSignInCountDay(value2.getSignInCount());
+            result.setAddWishlistCountTotal(value2.getAddWishlistCount());
+            result.setAddWishlistCountDay(value2.getAddWishlistCount());
+            result.setViewCommodityPageUvTotal(value2.getViewCommodityPageUv());
+            result.setViewCommodityPageUvDay(value2.getViewCommodityPageUv());
+            result.setPageReservationCountTotal(value2.getPageReservationCount());
+            result.setPageReservationCountDay(value2.getPageReservationCount());
+            result.setLeadsPurchaseUvTotal(value2.getLeadsPurchaseUv());
+            result.setLeadsPurchaseUvDay(value2.getLeadsPurchaseUv());
+            result.setLeadsPurchaseCostAll(value2.getLeadsPurchaseCost());
+            result.setLeadsPurchaseCostDay(value2.getLeadsPurchaseCost());
+            result.setLeadsPurchaseRateAll(value2.getLeadsPurchaseRate());
+            result.setLeadsPurchaseRateDay(value2.getLeadsPurchaseRate());
+            result.setScanFollowCountTotal(value2.getScanFollowCount());
+            result.setScanFollowCountDay(value2.getScanFollowCount());
+            result.setWechatAppRegisterUvTotal(value2.getWechatAppRegisterUv());
+            result.setWechatAppRegisterUvDay(value2.getWechatAppRegisterUv());
+            result.setWechatMinigameRegisterCostTotal(value2.getWechatMinigameRegisterCost());
+            result.setWechatMinigameRegisterCostDay(value2.getWechatMinigameRegisterCost());
+            result.setWechatMinigameRegisterRateAll(value2.getWechatMinigameRegisterRate());
+            result.setWechatMinigameRegisterRateDay(value2.getWechatMinigameRegisterRate());
+            result.setWechatMinigameArpuAll(value2.getWechatMinigameArpu());
+            result.setWechatMinigameArpuDay(value2.getWechatMinigameArpu());
+            result.setWechatMinigameRetentionCountTotal(value2.getWechatMinigameRetentionCount());
+            result.setWechatMinigameRetentionCountDay(value2.getWechatMinigameRetentionCount());
+            result.setWechatMinigameCheckoutCountTotal(value2.getWechatMinigameCheckoutCount());
+            result.setWechatMinigameCheckoutCountDay(value2.getWechatMinigameCheckoutCount());
+            result.setWechatMinigameCheckoutAmountTotal(value2.getWechatMinigameCheckoutAmount());
+            result.setWechatMinigameCheckoutAmountDay(value2.getWechatMinigameCheckoutAmount());
+            result.setOfficialAccountFollowCountTotal(value2.getOfficialAccountFollowCount());
+            result.setOfficialAccountFollowCountDay(value2.getOfficialAccountFollowCount());
+            result.setOfficialAccountFollowCostAll(value2.getOfficialAccountFollowCost());
+            result.setOfficialAccountFollowCostDay(value2.getOfficialAccountFollowCost());
+            result.setOfficialAccountFollowRateAll(value2.getOfficialAccountFollowRate());
+            result.setOfficialAccountFollowRateDay(value2.getOfficialAccountFollowRate());
+            result.setOfficialAccountRegisterUserCountTotal(value2.getOfficialAccountRegisterUserCount());
+            result.setOfficialAccountRegisterUserCountDay(value2.getOfficialAccountRegisterUserCount());
+            result.setOfficialAccountRegisterRateAll(value2.getOfficialAccountRegisterRate());
+            result.setOfficialAccountRegisterRateDay(value2.getOfficialAccountRegisterRate());
+            result.setOfficialAccountRegisterCostAll(value2.getOfficialAccountRegisterCost());
+            result.setOfficialAccountRegisterCostDay(value2.getOfficialAccountRegisterCost());
+            result.setOfficialAccountRegisterAmountTotal(value2.getOfficialAccountRegisterAmount());
+            result.setOfficialAccountRegisterAmountDay(value2.getOfficialAccountRegisterAmount());
+            result.setOfficialAccountRegisterRoiAll(value2.getOfficialAccountRegisterRoi());
+            result.setOfficialAccountRegisterRoiDay(value2.getOfficialAccountRegisterRoi());
+            result.setOfficialAccountApplyCountTotal(value2.getOfficialAccountApplyCount());
+            result.setOfficialAccountApplyCountDay(value2.getOfficialAccountApplyCount());
+            result.setOfficialAccountApplyUserCountTotal(value2.getOfficialAccountApplyUserCount());
+            result.setOfficialAccountApplyUserCountDay(value2.getOfficialAccountApplyUserCount());
+            result.setOfficialAccountApplyRateAll(value2.getOfficialAccountApplyRate());
+            result.setOfficialAccountApplyRateDay(value2.getOfficialAccountApplyRate());
+            result.setOfficialAccountApplyCostAll(value2.getOfficialAccountApplyCost());
+            result.setOfficialAccountApplyCostDay(value2.getOfficialAccountApplyCost());
+            result.setOfficialAccountApplyAmountTotal(value2.getOfficialAccountApplyAmount());
+            result.setOfficialAccountApplyAmountDay(value2.getOfficialAccountApplyAmount());
+            result.setOfficialAccountApplyRoiAll(value2.getOfficialAccountApplyRoi());
+            result.setOfficialAccountApplyRoiDay(value2.getOfficialAccountApplyRoi());
+            result.setOfficialAccountOrderCountTotal(value2.getOfficialAccountOrderCount());
+            result.setOfficialAccountOrderCountDay(value2.getOfficialAccountOrderCount());
+            result.setOfficialAccountFirstDayOrderCountTotal(value2.getOfficialAccountFirstDayOrderCount());
+            result.setOfficialAccountFirstDayOrderCountDay(value2.getOfficialAccountFirstDayOrderCount());
+            result.setOfficialAccountOrderUserCountTotal(value2.getOfficialAccountOrderUserCount());
+            result.setOfficialAccountOrderUserCountDay(value2.getOfficialAccountOrderUserCount());
+            result.setOfficialAccountOrderRateAll(value2.getOfficialAccountOrderRate());
+            result.setOfficialAccountOrderRateDay(value2.getOfficialAccountOrderRate());
+            result.setOfficialAccountOrderCostAll(value2.getOfficialAccountOrderCost());
+            result.setOfficialAccountOrderCostDay(value2.getOfficialAccountOrderCost());
+            result.setOfficialAccountOrderAmountTotal(value2.getOfficialAccountOrderAmount());
+            result.setOfficialAccountOrderAmountDay(value2.getOfficialAccountOrderAmount());
+            result.setOfficialAccountFirstDayOrderAmountTotal(value2.getOfficialAccountFirstDayOrderAmount());
+            result.setOfficialAccountFirstDayOrderAmountDay(value2.getOfficialAccountFirstDayOrderAmount());
+            result.setOfficialAccountOrderRoiAll(value2.getOfficialAccountOrderRoi());
+            result.setOfficialAccountOrderRoiDay(value2.getOfficialAccountOrderRoi());
+            result.setOfficialAccountConsultCountTotal(value2.getOfficialAccountConsultCount());
+            result.setOfficialAccountConsultCountDay(value2.getOfficialAccountConsultCount());
+            result.setOfficialAccountReaderCountTotal(value2.getOfficialAccountReaderCount());
+            result.setOfficialAccountReaderCountDay(value2.getOfficialAccountReaderCount());
+            result.setOfficialAccountCreditApplyUserCountTotal(value2.getOfficialAccountCreditApplyUserCount());
+            result.setOfficialAccountCreditApplyUserCountDay(value2.getOfficialAccountCreditApplyUserCount());
+            result.setOfficialAccountCreditUserCountTotal(value2.getOfficialAccountCreditUserCount());
+            result.setOfficialAccountCreditUserCountDay(value2.getOfficialAccountCreditUserCount());
+            result.setForwardCountTotal(value2.getForwardCount());
+            result.setForwardCountDay(value2.getForwardCount());
+            result.setForwardUserCountTotal(value2.getForwardUserCount());
+            result.setForwardUserCountDay(value2.getForwardUserCount());
+            result.setNoInterestCountTotal(value2.getNoInterestCount());
+            result.setNoInterestCountDay(value2.getNoInterestCount());
+        } else {
+            result.setCostDeviationRateTotal(value1.getCostDeviationRateTotal() + value2.getCostDeviationRate());
+            result.setCostDeviationRateDay(value2.getCostDeviationRate());
+            result.setCostTotal(value1.getCostTotal() + value2.getCost());
+            result.setCostDay(value2.getCost());
+            result.setCompensationAmountTotal(value1.getCompensationAmountTotal() + value2.getCompensationAmount());
+            result.setCompensationAmountDay(value2.getCompensationAmount());
+            result.setViewCountTotal(value1.getViewCountTotal() + value2.getViewCount());
+            result.setViewCountDay(value2.getViewCount());
+            // 总消耗 / 总曝光
+            result.setThousandDisplayPriceAll(result.getViewCountTotal() == 0 ? 0 : (result.getCostTotal() / result.getViewCountTotal() * 1000));
+            result.setThousandDisplayPriceDay(value2.getThousandDisplayPrice());
+
+            // 应该不准,因为跨天的用户无法去重
+            result.setViewUserCountTotal(value1.getViewUserCountTotal() + value2.getViewUserCount());
+            result.setViewUserCountDay(value2.getViewUserCount());
+            // 曝光次数 / 曝光人数(不准)
+            result.setAvgViewPerUserAll(result.getViewUserCountTotal() == 0 ? 0.0 : result.getViewCountTotal() / result.getViewUserCountTotal());
+            result.setAvgViewPerUserDay(value2.getAvgViewPerUser());
+            result.setValidClickCountTotal(value1.getValidClickCountTotal() + value2.getValidClickCount());
+            result.setValidClickCountDay(value2.getValidClickCount());
+
+            // 应该不准,因为跨天的用户无法去重
+            result.setClickUserCountTotal(value1.getClickUserCountTotal() + value2.getClickUserCount());
+            result.setClickUserCountDay(value2.getClickUserCount());
+            // 广告点击次数 / 广告曝光次数
+            result.setCtrAll(result.getViewCountTotal() == 0 ? 0.0 : result.getValidClickCountTotal() / result.getViewCountTotal());
+            result.setCtrDay(value2.getCtr());
+            // 广告花费/广告点击次数
+            result.setCpcAll(result.getValidClickCountTotal() == 0 ? 0 : result.getCostTotal() / result.getValidClickCountTotal());
+            result.setCpcDay(value2.getCpc());
+            result.setValuableClickCountTotal(value1.getValuableClickCountTotal() + value2.getValuableClickCount());
+            result.setValuableClickCountDay(value2.getValuableClickCount());
+            // 广告可转化点击次数/广告曝光次数
+            result.setValuableClickRateAll(result.getViewCountTotal() == 0 ? 0.0 : result.getValuableClickCountTotal() / result.getViewCountTotal());
+            result.setValuableClickRateDay(value2.getValuableClickRate());
+            // 广告花费/可转化点击次数
+            result.setValuableClickCostAll(result.getValuableClickCountTotal() == 0 ? 0 : result.getCostTotal() / result.getValuableClickCountTotal());
+            result.setValuableClickCostDay(value2.getValuableClickCost());
+            result.setConversionsCountTotal(value1.getConversionsCountTotal() + value2.getConversionsCount());
+            result.setConversionsCountDay(value2.getConversionsCount());
+            // 广告花费/转化目标量
+            result.setConversionsCostAll(result.getConversionsCountTotal() == 0 ? 0 : result.getCostTotal() / result.getConversionsCountTotal());
+            result.setConversionsCostDay(value2.getConversionsCost());
+            // 公众号:转化目标量/点击次数。
+            result.setConversionsRateAll(result.getValidClickCountTotal() == 0 ? 0.0 : result.getConversionsCountTotal() / result.getValidClickCountTotal());
+            result.setConversionsRateDay(value2.getConversionsRate());
+            result.setDeepConversionsCountTotal(value1.getDeepConversionsCountTotal() + value2.getDeepConversionsCount());
+            result.setDeepConversionsCountDay(value2.getDeepConversionsCount());
+            // 广告花费/深度转化目标量
+            result.setDeepConversionsCostTotal(result.getDeepConversionsCountTotal() == 0 ? 0 : result.getCostTotal() / result.getDeepConversionsCountTotal());
+            result.setDeepConversionsCostDay(value2.getDeepConversionsCost());
+            // 深度转化目标量/可转化点击次数
+            result.setDeepConversionsRateTotal(result.getValuableClickCountTotal() == 0 ? 0.0 : result.getDeepConversionsCountTotal() / result.getValuableClickCountTotal());
+            result.setDeepConversionsRateDay(value2.getDeepConversionsRate());
+            result.setKeyPageUvTotal(value1.getKeyPageUvTotal() + value2.getKeyPageUv());
+            result.setKeyPageUvDay(value2.getKeyPageUv());
+            result.setOrderCountTotal(value1.getOrderCountTotal() + value2.getOrderCount());
+            result.setOrderCountDay(value2.getOrderCount());
+            result.setFirstDayOrderCountTotal(value1.getFirstDayOrderCountTotal() + value2.getFirstDayOrderCount());
+            result.setFirstDayOrderCountDay(value2.getFirstDayOrderCount());
+            // 广告花费/下单量
+            result.setWebOrderCostAll(result.getOrderCountTotal() == 0 ? 0 : result.getCostTotal() / result.getOrderCountTotal());
+            result.setWebOrderCostDay(value2.getWebOrderCost());
+            // 下单量/点击次数
+            result.setOrderRateAll(result.getValidClickCountTotal() == 0 ? 0.0 : result.getOrderCountTotal() / result.getValidClickCountTotal());
+            result.setOrderRateDay(value2.getOrderRate());
+            result.setOrderAmountTotal(value1.getOrderAmountTotal() + value2.getOrderAmount());
+            result.setOrderAmountDay(value2.getOrderAmount());
+            result.setFirstDayOrderAmountTotal(value1.getFirstDayOrderAmountTotal() + value2.getFirstDayOrderAmount());
+            result.setFirstDayOrderAmountDay(value2.getFirstDayOrderAmount());
+            // 下单金额/下单量
+            result.setOrderUnitPriceAll(result.getOrderCountTotal() == 0 ? 0 : result.getOrderAmountTotal() / result.getOrderCountTotal());
+            result.setOrderUnitPriceDay(value2.getOrderUnitPrice());
+            // 下单金额/广告花费
+            result.setOrderRoiAll(result.getCostTotal() == 0 ? 0.0 : result.getOrderAmountTotal() / result.getCostTotal());
+            result.setOrderRoiDay(value2.getOrderRoi());
+            result.setSignInCountTotal(value1.getSignInCountTotal() + value2.getSignInCount());
+            result.setSignInCountDay(value2.getSignInCount());
+            result.setAddWishlistCountTotal(value1.getAddWishlistCountTotal() + value2.getAddWishlistCount());
+            result.setAddWishlistCountDay(value2.getAddWishlistCount());
+            result.setViewCommodityPageUvTotal(value1.getViewCommodityPageUvTotal() + value2.getViewCommodityPageUv());
+            result.setViewCommodityPageUvDay(value2.getViewCommodityPageUv());
+            result.setPageReservationCountTotal(value1.getPageReservationCountTotal() + value2.getPageReservationCount());
+            result.setPageReservationCountDay(value2.getPageReservationCount());
+
+            // 应该不准,因为跨天的用户无法去重
+            result.setLeadsPurchaseUvTotal(value1.getLeadsPurchaseUvTotal() + value2.getLeadsPurchaseUv());
+            result.setLeadsPurchaseUvDay(value2.getLeadsPurchaseUv());
+            // 广告消耗/首次付费人数
+            result.setLeadsPurchaseCostAll(result.getLeadsPurchaseUvTotal() == 0 ? 0 : result.getCostTotal() / result.getLeadsPurchaseUvTotal());
+            result.setLeadsPurchaseCostDay(value2.getLeadsPurchaseCost());
+            // 首次付费人数/广告点击次数
+            result.setLeadsPurchaseRateAll(result.getValidClickCountTotal() == 0 ? 0.0 : result.getLeadsPurchaseUvTotal() / result.getValidClickCountTotal());
+            result.setLeadsPurchaseRateDay(value2.getLeadsPurchaseRate());
+            result.setScanFollowCountTotal(value1.getScanFollowCountTotal() + value2.getScanFollowCount());
+            result.setScanFollowCountDay(value2.getScanFollowCount());
+            result.setWechatAppRegisterUvTotal(value1.getWechatAppRegisterUvTotal() + value2.getWechatAppRegisterUv());
+            result.setWechatAppRegisterUvDay(value2.getWechatAppRegisterUv());
+            // 广告消耗 / 小游戏注册人数
+            result.setWechatMinigameRegisterCostTotal(result.getWechatAppRegisterUvTotal() == 0 ? 0 : result.getCostTotal() / result.getWechatAppRegisterUvTotal());
+            result.setWechatMinigameRegisterCostDay(value2.getWechatMinigameRegisterCost());
+            // 小游戏注册人数 / 广告点击次数
+            result.setWechatMinigameRegisterRateAll(result.getValidClickCountTotal() == 0 ? 0.0 : result.getWechatAppRegisterUvTotal() / result.getValidClickCountTotal());
+            result.setWechatMinigameRegisterRateDay(value2.getWechatMinigameRegisterRate());
+
+            //------------------未知
+            result.setWechatMinigameArpuAll(value1.getWechatMinigameArpuAll() + value2.getWechatMinigameArpu());
+            result.setWechatMinigameArpuDay(value2.getWechatMinigameArpu());
+            result.setWechatMinigameRetentionCountTotal(value1.getWechatMinigameRetentionCountTotal() + value2.getWechatMinigameRetentionCount());
+            result.setWechatMinigameRetentionCountDay(value2.getWechatMinigameRetentionCount());
+            result.setWechatMinigameCheckoutCountTotal(value1.getWechatMinigameCheckoutCountTotal() + value2.getWechatMinigameCheckoutCount());
+            result.setWechatMinigameCheckoutCountDay(value2.getWechatMinigameCheckoutCount());
+            result.setWechatMinigameCheckoutAmountTotal(value1.getWechatMinigameCheckoutAmountTotal() + value2.getWechatMinigameCheckoutAmount());
+            result.setWechatMinigameCheckoutAmountDay(value2.getWechatMinigameCheckoutAmount());
+            result.setOfficialAccountFollowCountTotal(value1.getOfficialAccountFollowCountTotal() + value2.getOfficialAccountFollowCount());
+            result.setOfficialAccountFollowCountDay(value2.getOfficialAccountFollowCount());
+            // 广告消耗 / 关注次数
+            result.setOfficialAccountFollowCostAll(result.getOfficialAccountFollowCountTotal() == 0 ? 0 : result.getCostTotal() / result.getOfficialAccountFollowCountTotal());
+            result.setOfficialAccountFollowCostDay(value2.getOfficialAccountFollowCost());
+            // 关注次数 / 点击次数
+            result.setOfficialAccountFollowRateAll(result.getValidClickCountTotal() == 0 ? 0.0 : result.getOfficialAccountFollowCountTotal() / result.getValidClickCountTotal());
+            result.setOfficialAccountFollowRateDay(value2.getOfficialAccountFollowRate());
+            result.setOfficialAccountRegisterUserCountTotal(value1.getOfficialAccountRegisterUserCountTotal() + value2.getOfficialAccountRegisterUserCount());
+            result.setOfficialAccountRegisterUserCountDay(value2.getOfficialAccountRegisterUserCount());
+            // 公众号内注册人数 / 公众号关注次数
+            result.setOfficialAccountRegisterRateAll(result.getOfficialAccountFollowCountTotal() == 0 ? 0.0 : result.getOfficialAccountRegisterUserCountTotal() / result.getOfficialAccountFollowCountTotal());
+            result.setOfficialAccountRegisterRateDay(value2.getOfficialAccountRegisterRate());
+            // 广告消耗 / 广告注册人数
+            result.setOfficialAccountRegisterCostAll(result.getOfficialAccountRegisterUserCountTotal() == 0 ? 0 : result.getCostTotal() / result.getOfficialAccountRegisterUserCountTotal());
+            result.setOfficialAccountRegisterCostDay(value2.getOfficialAccountRegisterCost());
+            result.setOfficialAccountRegisterAmountTotal(value1.getOfficialAccountRegisterAmountTotal() + value2.getOfficialAccountRegisterAmount());
+            result.setOfficialAccountRegisterAmountDay(value2.getOfficialAccountRegisterAmount());
+            // 注册产生的订单金额累计/广告花费
+            result.setOfficialAccountRegisterRoiAll(result.getCostTotal() == 0 ? 0 : result.getOfficialAccountRegisterAmountTotal() / result.getCostTotal());
+            result.setOfficialAccountRegisterRoiDay(value2.getOfficialAccountRegisterRoi());
+            result.setOfficialAccountApplyCountTotal(value1.getOfficialAccountApplyCountTotal() + value2.getOfficialAccountApplyCount());
+            result.setOfficialAccountApplyCountDay(value2.getOfficialAccountApplyCount());
+            result.setOfficialAccountApplyUserCountTotal(value1.getOfficialAccountApplyUserCountTotal() + value2.getOfficialAccountApplyUserCount());
+            result.setOfficialAccountApplyUserCountDay(value2.getOfficialAccountApplyUserCount());
+            // 公众号内填单的独立用户数/公众号关注次数
+            result.setOfficialAccountApplyRateAll(result.getOfficialAccountFollowCountTotal() == 0 ? 0.0 : result.getOfficialAccountApplyUserCountTotal() / result.getOfficialAccountFollowCountTotal());
+            result.setOfficialAccountApplyRateDay( value2.getOfficialAccountApplyRate());
+            // 广告花费/广告产生的填单行为数量
+            result.setOfficialAccountApplyCostAll(result.getOfficialAccountApplyUserCountTotal() == 0 ? 0 : result.getCostTotal() / result.getOfficialAccountApplyUserCountTotal());
+            result.setOfficialAccountApplyCostDay(value2.getOfficialAccountApplyCost());
+            result.setOfficialAccountApplyAmountTotal(value1.getOfficialAccountApplyAmountTotal() + value2.getOfficialAccountApplyAmount());
+            result.setOfficialAccountApplyAmountDay(value2.getOfficialAccountApplyAmount());
+            // 填单产生的订单金额累计/广告花费
+            result.setOfficialAccountApplyRoiAll(result.getCostTotal() == 0 ? 0 : result.getOfficialAccountApplyAmountTotal() / result.getCostTotal());
+            result.setOfficialAccountApplyRoiDay(value2.getOfficialAccountApplyRoi());
+            result.setOfficialAccountOrderCountTotal(value1.getOfficialAccountOrderCountTotal() + value2.getOfficialAccountOrderCount());
+            result.setOfficialAccountOrderCountDay(value2.getOfficialAccountOrderCount());
+            result.setOfficialAccountFirstDayOrderCountTotal(value1.getOfficialAccountFirstDayOrderCountTotal() + value2.getOfficialAccountFirstDayOrderCount());
+            result.setOfficialAccountFirstDayOrderCountDay(value2.getOfficialAccountFirstDayOrderCount());
+            result.setOfficialAccountOrderUserCountTotal(value1.getOfficialAccountOrderUserCountTotal() + value2.getOfficialAccountOrderUserCount());
+            result.setOfficialAccountOrderUserCountDay(value2.getOfficialAccountOrderUserCount());
+            // 公众号内下单独立用户数(UV)/公众号关注次数
+            result.setOfficialAccountOrderRateAll(result.getOfficialAccountFollowCountTotal() == 0 ? 0.0 : result.getOfficialAccountOrderUserCountTotal() / result.getOfficialAccountFollowCountTotal());
+            result.setOfficialAccountOrderRateDay(value2.getOfficialAccountOrderRate());
+            // 广告花费/广告产生的下单行为数量
+            result.setOfficialAccountOrderCostAll(result.getOfficialAccountOrderCountTotal() == 0 ? 0 : result.getCostTotal() / result.getOfficialAccountOrderCountTotal());
+            result.setOfficialAccountOrderCostDay(value2.getOfficialAccountOrderCost());
+            result.setOfficialAccountOrderAmountTotal(value1.getOfficialAccountOrderAmountTotal() + value2.getOfficialAccountOrderAmount());
+            result.setOfficialAccountOrderAmountDay(value2.getOfficialAccountOrderAmount());
+            result.setOfficialAccountFirstDayOrderAmountTotal(value1.getOfficialAccountFirstDayOrderAmountTotal() + value2.getOfficialAccountFirstDayOrderAmount());
+            result.setOfficialAccountFirstDayOrderAmountDay(value2.getOfficialAccountFirstDayOrderAmount());
+            // 下单产生的订单金额累计/广告花费
+            result.setOfficialAccountOrderRoiAll(result.getCostTotal() == 0 ? 0 : result.getOfficialAccountOrderAmountTotal() / result.getCostTotal());
+            result.setOfficialAccountOrderRoiDay(value2.getOfficialAccountOrderRoi());
+            result.setOfficialAccountConsultCountTotal(value1.getOfficialAccountConsultCountTotal() + value2.getOfficialAccountConsultCount());
+            result.setOfficialAccountConsultCountDay(value2.getOfficialAccountConsultCount());
+            result.setOfficialAccountReaderCountTotal(value1.getOfficialAccountReaderCountTotal() + value2.getOfficialAccountReaderCount());
+            result.setOfficialAccountReaderCountDay(value2.getOfficialAccountReaderCount());
+            result.setOfficialAccountCreditApplyUserCountTotal(value1.getOfficialAccountCreditApplyUserCountTotal() + value2.getOfficialAccountCreditApplyUserCount());
+            result.setOfficialAccountCreditApplyUserCountDay(value2.getOfficialAccountCreditApplyUserCount());
+            result.setOfficialAccountCreditUserCountTotal(value1.getOfficialAccountCreditUserCountTotal() + value2.getOfficialAccountCreditUserCount());
+            result.setOfficialAccountCreditUserCountDay(value2.getOfficialAccountCreditUserCount());
+            result.setForwardCountTotal(value1.getForwardCountTotal() + value2.getForwardCount());
+            result.setForwardCountDay(value2.getForwardCount());
+            result.setForwardUserCountTotal(value1.getForwardUserCountTotal() + value2.getForwardUserCount());
+            result.setForwardUserCountDay(value2.getForwardUserCount());
+            result.setNoInterestCountTotal(value1.getNoInterestCountTotal() + value2.getNoInterestCount());
+            result.setNoInterestCountDay(value2.getNoInterestCount());
+        }
+        return result;
+    }
 }