Ver Fonte

优化设备告警逻辑

zhou-hao há 3 anos atrás
pai
commit
1b29bd5e9f

+ 35 - 41
jetlinks-manager/rule-engine-manager/src/main/java/org/jetlinks/community/rule/engine/device/DeviceAlarmRule.java

@@ -4,6 +4,7 @@ import io.swagger.v3.oas.annotations.Hidden;
 import io.swagger.v3.oas.annotations.media.Schema;
 import lombok.AllArgsConstructor;
 import lombok.Getter;
+import lombok.NoArgsConstructor;
 import lombok.Setter;
 import org.jetlinks.community.rule.engine.model.Action;
 import org.jetlinks.core.message.DeviceMessage;
@@ -92,6 +93,14 @@ public class DeviceAlarmRule implements Serializable {
     @Schema(description = "防抖限制")
     private ShakeLimit shakeLimit;
 
+    @Schema(description = "告警级别")
+    @Hidden
+    private Integer level;
+
+    @Schema(description = "告警类别")
+    @Hidden
+    private String type;
+
     public void validate() {
         if (org.apache.commons.collections.CollectionUtils.isEmpty(getTriggers())) {
             throw new IllegalArgumentException("触发条件不能为空");
@@ -208,7 +217,7 @@ public class DeviceAlarmRule implements Serializable {
         private List<FunctionParameter> parameters;
 
         //物模型属性或者事件的标识 如: fire_alarm
-        @Schema(description = "物模型表示,如:属性ID,事件ID")
+        @Schema(description = "物模型标识,如:属性ID,事件ID")
         private String modelId;
 
         //过滤条件
@@ -218,21 +227,21 @@ public class DeviceAlarmRule implements Serializable {
         public Set<String> toColumns() {
 
             return Stream.concat(
-                (StringUtils.hasText(modelId)
-                    ? Collections.singleton(type.getPropertyPrefix() + "this['" + modelId + "'] \"" + modelId + "\"")
-                    : Collections.<String>emptySet()).stream(),
-                (CollectionUtils.isEmpty(filters)
-                    ? Stream.<ConditionFilter>empty()
-                    : filters.stream())
-                    .map(filter -> filter.getColumn(type)))
-                .collect(Collectors.toSet());
+                             (StringUtils.hasText(modelId)
+                                 ? Collections.singleton(type.getPropertyPrefix() + "this['" + modelId + "'] \"" + modelId + "\"")
+                                 : Collections.<String>emptySet()).stream(),
+                             (CollectionUtils.isEmpty(filters)
+                                 ? Stream.<ConditionFilter>empty()
+                                 : filters.stream())
+                                 .map(filter -> filter.getColumn(type)))
+                         .collect(Collectors.toSet());
         }
 
         public List<Object> toFilterBinds() {
             return filters == null ? Collections.emptyList() :
                 filters.stream()
-                    .map(ConditionFilter::convertValue)
-                    .collect(Collectors.toList());
+                       .map(ConditionFilter::convertValue)
+                       .collect(Collectors.toList());
         }
 
         public Optional<String> createExpression() {
@@ -241,8 +250,8 @@ public class DeviceAlarmRule implements Serializable {
             }
             return Optional.of(
                 filters.stream()
-                    .map(filter -> filter.createExpression(type))
-                    .collect(Collectors.joining(" and "))
+                       .map(filter -> filter.createExpression(type))
+                       .collect(Collectors.joining(" and "))
             );
         }
 
@@ -271,34 +280,10 @@ public class DeviceAlarmRule implements Serializable {
         }
     }
 
-    /**
-     * 抖动限制
-     * <a href="https://github.com/jetlinks/jetlinks-community/issues/8">https://github.com/jetlinks/jetlinks-community/issues/8</a>
-     *
-     * @since 1.3
-     */
-    @Getter
-    @Setter
-    public static class ShakeLimit implements Serializable {
-        @Schema(description = "是否开启防抖")
-        private boolean enabled;
-
-        //时间限制,单位时间内发生多次告警时,只算一次。单位:秒
-        @Schema(description = "时间间隔(秒)")
-        private int time;
-
-        //触发阈值,单位时间内发生n次告警,只算一次。
-        @Schema(description = "触发阈值(次)")
-        private int threshold;
-
-        //当发生第一次告警时就触发,为false时表示最后一次才触发(告警有延迟,但是可以统计出次数)
-        @Schema(description = "是否第一次满足条件就触发")
-        private boolean alarmFirst;
-
-    }
-
     @Getter
     @Setter
+    @AllArgsConstructor
+    @NoArgsConstructor
     public static class ConditionFilter implements Serializable {
         //过滤条件key 如: temperature
         @Schema(description = "条件key")
@@ -317,11 +302,20 @@ public class DeviceAlarmRule implements Serializable {
         }
 
         public String createExpression(MessageType type) {
+            return createExpression(type, true);
+        }
+
+        public String createExpression(MessageType type, boolean prepareSQL) {
             //函数和this忽略前缀
             if (key.contains("(") || key.startsWith("this")) {
-                return key;
+                return key + operator.symbol + " ? ";
             }
-            return type.getPropertyPrefix() + "this['" + (key.trim()) + "'] " + operator.symbol + " ? ";
+            return type.getPropertyPrefix() + "this['" + (key.trim()) + "'] " + operator.symbol
+                + (prepareSQL ? " ? " : valueIsExpression() ? value : "'" + value + "'");
+        }
+
+        public boolean valueIsExpression() {
+            return false;
         }
 
         public Object convertValue() {

+ 80 - 45
jetlinks-manager/rule-engine-manager/src/main/java/org/jetlinks/community/rule/engine/device/DeviceAlarmTaskExecutorProvider.java

@@ -26,6 +26,7 @@ import reactor.core.Disposable;
 import reactor.core.publisher.Flux;
 import reactor.core.publisher.Mono;
 import reactor.core.scheduler.Scheduler;
+import reactor.core.scheduler.Schedulers;
 import reactor.util.function.Tuples;
 
 import java.time.Duration;
@@ -47,21 +48,28 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
 
     @Override
     public Mono<TaskExecutor> createTask(ExecutionContext context) {
-        return Mono.just(new DeviceAlarmTaskExecutor(context));
+        return Mono.just(new DeviceAlarmTaskExecutor(context, eventBus, scheduler));
     }
 
-    class DeviceAlarmTaskExecutor extends AbstractTaskExecutor {
+    static class DeviceAlarmTaskExecutor extends AbstractTaskExecutor {
 
         List<String> default_columns = Arrays.asList(
-            "timestamp", "deviceId", "this.headers.deviceName deviceName"
+            "timestamp", "deviceId", "this.headers headers", "this.headers.deviceName deviceName"
         );
+        private final EventBus eventBus;
+
+        private final Scheduler scheduler;
 
         private DeviceAlarmRule rule;
 
         private ReactorQL ql;
 
-        public DeviceAlarmTaskExecutor(ExecutionContext context) {
+        DeviceAlarmTaskExecutor(ExecutionContext context,
+                                EventBus eventBus,
+                                Scheduler scheduler) {
             super(context);
+            this.eventBus = eventBus;
+            this.scheduler = scheduler;
             rule = createRule();
             ql = createQL(rule);
         }
@@ -77,7 +85,7 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
             return doSubscribe(eventBus)
                 .filter(ignore -> state == Task.State.running)
                 .flatMap(result -> {
-                    RuleData data = RuleData.create(result);
+                    RuleData data = context.newRuleData(result);
                     //输出到下一节点
                     return context
                         .getOutput()
@@ -99,9 +107,11 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
         }
 
         private DeviceAlarmRule createRule() {
-            DeviceAlarmRule rule = ValueObject.of(context.getJob().getConfiguration())
+            DeviceAlarmRule rule = ValueObject
+                .of(context.getJob().getConfiguration())
                 .get("rule")
-                .map(val -> FastBeanCopier.copy(val, new DeviceAlarmRule())).orElseThrow(() -> new IllegalArgumentException("告警配置错误"));
+                .map(val -> FastBeanCopier.copy(val, new DeviceAlarmRule()))
+                .orElseThrow(() -> new IllegalArgumentException("告警配置错误"));
             rule.validate();
             return rule;
         }
@@ -128,7 +138,7 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                 columns.add(trigger.getType().getPropertyPrefix() + "this trigger" + i);
                 columns.addAll(trigger.toColumns());
                 trigger.createExpression()
-                    .ifPresent(expr -> wheres.add("(" + expr + ")"));
+                       .ifPresent(expr -> wheres.add("(" + expr + ")"));
             }
             String sql = "select \n\t\t" + String.join("\n\t\t,", columns) + " \n\tfrom dual ";
 
@@ -140,6 +150,7 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                 List<String> newColumns = new ArrayList<>(Arrays.asList(
                     "this.deviceName deviceName",
                     "this.deviceId deviceId",
+                    "this.headers headers",
                     "this.timestamp timestamp"));
                 for (DeviceAlarmRule.Property property : rule.getProperties()) {
                     if (StringUtils.isEmpty(property.getProperty())) {
@@ -155,7 +166,7 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                         newColumns.add("this['" + property.getProperty() + "'] \"" + alias + "\"");
                     }
                 }
-                if (newColumns.size() > 3) {
+                if (newColumns.size() > 4) {
                     sql = "select \n\t" + String.join("\n\t,", newColumns) + "\n from (\n\t" + sql + "\n) t";
                 }
             }
@@ -170,19 +181,36 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
             List<Object> binds = new ArrayList<>();
 
             for (DeviceAlarmRule.Trigger trigger : rule.getTriggers()) {
-                String topic = trigger.getType().getTopic(rule.getProductId(), rule.getDeviceId(), trigger.getModelId());
-                topics.add(topic);
                 binds.addAll(trigger.toFilterBinds());
+                //since 1.11 定时触发的不从eventBus订阅
+                if (trigger.getTrigger() == DeviceAlarmRule.TriggerType.timer) {
+                    continue;
+                }
+
+                String topic = trigger
+                    .getType()
+                    .getTopic(rule.getProductId(), rule.getDeviceId(), trigger.getModelId());
+                topics.add(topic);
             }
-            Subscription subscription = Subscription.of(
-                "device_alarm:" + rule.getId(),
-                topics.toArray(new String[0]),
-                Subscription.Feature.local
+
+            List<Flux<? extends Map<String, Object>>> inputs = new ArrayList<>();
+
+            //从上游获取输入进行处理(通常是定时触发发送指令后得到的回复)
+            inputs.add(
+                context
+                    .getInput()
+                    .accept()
+                    .flatMap(RuleData::dataToMap)
             );
-//            List<Subscription> subscriptions = topics.stream().map(Subscription::new).collect(Collectors.toList());
 
-            ReactorQLContext context = ReactorQLContext
-                .ofDatasource(ignore ->
+            //从事件总线订阅数据进行处理
+            if (!topics.isEmpty()) {
+                Subscription subscription = Subscription.of(
+                    "device_alarm:" + rule.getId(),
+                    topics.toArray(new String[0]),
+                    Subscription.Feature.local
+                );
+                inputs.add(
                     eventBus
                         .subscribe(subscription, DeviceMessage.class)
                         .map(Jsonable::toJson)
@@ -198,6 +226,9 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                             json.put("alarmName", rule.getName());
                         })
                 );
+            }
+            ReactorQLContext context = ReactorQLContext
+                .ofDatasource(ignore -> Flux.merge(inputs));
 
             binds.forEach(context::bind);
 
@@ -205,36 +236,36 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                 .start(context)
                 .map(ReactorQLRecord::asMap);
 
-            DeviceAlarmRule.ShakeLimit shakeLimit;
-            if ((shakeLimit = rule.getShakeLimit()) != null
-                && shakeLimit.isEnabled()
-                && shakeLimit.getTime() > 0) {
-                int thresholdNumber = shakeLimit.getThreshold();
-                Duration windowTime = Duration.ofSeconds(shakeLimit.getTime());
+            ShakeLimit shakeLimit;
+            if ((shakeLimit = rule.getShakeLimit()) != null) {
 
-                resultFlux = resultFlux
-                    .as(flux ->
+                resultFlux = shakeLimit.transfer(
+                    resultFlux,
+                    (duration, flux) ->
                         StringUtils.hasText(rule.getDeviceId())
-                            ? flux.window(windowTime, scheduler)//规则已经指定了固定的设备,直接开启时间窗口就行
-                            : flux //规则配置在设备产品上,则按设备ID分组后再开窗口
+                            //规则已经指定了固定的设备,直接开启时间窗口就行
+                            ? flux.window(duration, scheduler)
+                            //规则配置在设备产品上,则按设备ID分组后再开窗口
+                            : flux
                             .groupBy(map -> String.valueOf(map.get("deviceId")), Integer.MAX_VALUE)
-                            .flatMap(group -> group.window(windowTime, scheduler), Integer.MAX_VALUE))
-                    //处理每一组数据
-                    .flatMap(group -> group
-                        .index((index, data) -> Tuples.of(index + 1, data)) //给数据打上索引,索引号就是告警次数
-                        .filter(tp -> tp.getT1() >= thresholdNumber)//超过阈值告警
-                        .as(flux -> shakeLimit.isAlarmFirst() ? flux.take(1) : flux.takeLast(1))//取第一个或者最后一个
-                        .map(tp2 -> {
-                            tp2.getT2().put("totalAlarms", tp2.getT1());
-                            return tp2.getT2();
-                        }));
+                            .flatMap(group -> group.window(duration, scheduler), Integer.MAX_VALUE),
+                    (alarm, total) -> alarm.put("totalAlarms", total)
+                );
             }
 
             return resultFlux
                 .flatMap(map -> {
+                    @SuppressWarnings("all")
+                    Map<String, Object> headers = (Map<String, Object>) map.remove("headers");
                     map.put("productId", rule.getProductId());
                     map.put("alarmId", rule.getId());
                     map.put("alarmName", rule.getName());
+                    if (null != rule.getLevel()) {
+                        map.put("alarmLevel", rule.getLevel());
+                    }
+                    if (null != rule.getType()) {
+                        map.put("alarmType", rule.getType());
+                    }
                     if (StringUtils.hasText(rule.getDeviceName())) {
                         map.putIfAbsent("deviceName", rule.getDeviceName());
                     }
@@ -244,26 +275,30 @@ public class DeviceAlarmTaskExecutorProvider implements TaskExecutorProvider {
                     if (StringUtils.hasText(rule.getDeviceId())) {
                         map.putIfAbsent("deviceId", rule.getDeviceId());
                     }
-                    if (!map.containsKey("deviceName")) {
+                    if (!map.containsKey("deviceName") && map.get("deviceId") != null) {
                         map.putIfAbsent("deviceName", map.get("deviceId"));
                     }
                     if (!map.containsKey("productName")) {
-                        map.putIfAbsent("productName", map.get("productId"));
+                        map.putIfAbsent("productName", rule.getProductId());
                     }
-                    //生成告警记录时生成ID,方便下游做处理。
-                    map.putIfAbsent("id", IDGenerator.MD5.generate());
                     if (log.isDebugEnabled()) {
                         log.debug("发生设备告警:{}", map);
                     }
+
+                    //生成告警记录时生成ID,方便下游做处理。
+                    map.putIfAbsent("id", IDGenerator.MD5.generate());
                     // 推送告警信息到消息网关中
                     // /rule-engine/device/alarm/{productId}/{deviceId}/{ruleId}
                     return eventBus
                         .publish(String.format(
                             "/rule-engine/device/alarm/%s/%s/%s",
-                            rule.getProductId(), map.get("deviceId"), rule.getId()
-                        ), map)
-                        .then(Mono.just(map));
+                            rule.getProductId(),
+                            map.get("deviceId"),
+                            rule.getId()), map)
+                        .thenReturn(map);
+
                 });
         }
     }
+
 }

+ 103 - 0
jetlinks-manager/rule-engine-manager/src/main/java/org/jetlinks/community/rule/engine/device/ShakeLimit.java

@@ -0,0 +1,103 @@
+package org.jetlinks.community.rule.engine.device;
+
+import io.swagger.v3.oas.annotations.media.Schema;
+import lombok.Getter;
+import lombok.Setter;
+import org.springframework.util.StringUtils;
+import reactor.core.publisher.Flux;
+import reactor.util.function.Tuples;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+
+/**
+ * 抖动限制
+ * <a href="https://github.com/jetlinks/jetlinks-community/issues/8">https://github.com/jetlinks/jetlinks-community/issues/8</a>
+ *
+ * @since 1.3
+ */
+@Getter
+@Setter
+public class ShakeLimit implements Serializable {
+    @Schema(description = "是否开启防抖")
+    private boolean enabled;
+
+    //时间限制,单位时间内发生多次告警时,只算一次。单位:秒
+    @Schema(description = "时间间隔(秒)")
+    private int time;
+
+    //触发阈值,单位时间内发生n次告警,只算一次。
+    @Schema(description = "触发阈值(次)")
+    private int threshold;
+
+    //当发生第一次告警时就触发,为false时表示最后一次才触发(告警有延迟,但是可以统计出次数)
+    @Schema(description = "是否第一次满足条件就触发")
+    private boolean alarmFirst;
+
+    /**
+     *
+     * 利用窗口函数,将ReactorQL语句包装为支持抖动限制的SQL.
+     *
+     * select * from ( sql )
+     * group by
+     * _window('1s') --时间窗口
+     * ,trace() -- 跟踪分组内行号信息
+     * ,take(-1) --取最后一条数据
+     * having row.index >= 2"; -- 行号信息索引就是数据量
+     *
+     * @param sql 原始SQL
+     * @return 防抖SQL
+     */
+    public String wrapReactorQl(@Nonnull String sql,
+                                @Nullable String groupBy) {
+        if (!enabled || time <= 0) {
+            return sql;
+        }
+        int takes = Math.max(threshold, 1);
+
+        return "select t.* from (" + sql + ") t" +
+            " group by " + (StringUtils.hasText(groupBy) ? groupBy + "," : "") +
+            "_window('" + time + "s')" + //时间窗口
+            ",trace()" +    //跟踪分组后新的行信息,row.index为分组内的行号,row.elapsed为与上一行数据间隔时间(毫秒)
+            ",take(" + (alarmFirst ? takes : -1) + ")" +
+            " having row.index >= " + takes;
+
+    }
+
+    /**
+     * 将流转换为支持抖动限制的流
+     *
+     * @param source         数据源
+     * @param windowFunction 窗口函数
+     * @param totalConsumer  总数接收器
+     * @param <T>            数据类型
+     * @return 新流
+     */
+    public <T> Flux<T> transfer(Flux<T> source,
+                                BiFunction<Duration, Flux<T>, Flux<Flux<T>>> windowFunction,
+                                BiConsumer<T, Long> totalConsumer) {
+        if (!enabled || time <= 0) {
+            return source;
+        }
+        int thresholdNumber = getThreshold();
+        Duration windowTime = Duration.ofSeconds(getTime());
+
+        return source
+            .as(flux -> windowFunction.apply(windowTime, flux))
+            //处理每一组数据
+            .flatMap(group -> group
+                //给数据打上索引,索引号就是告警次数
+                .index((index, data) -> Tuples.of(index + 1, data))
+                //超过阈值告警时
+                .filter(tp -> tp.getT1() >= thresholdNumber)
+                .as(flux -> isAlarmFirst() ? flux.take(1) : flux.takeLast(1))//取第一个或者最后一个
+                .map(tp2 -> {
+                    totalConsumer.accept(tp2.getT2(), tp2.getT1());
+                    return tp2.getT2();
+                }));
+    }
+}

+ 29 - 13
jetlinks-manager/rule-engine-manager/src/main/java/org/jetlinks/community/rule/engine/model/DeviceAlarmModelParser.java

@@ -17,6 +17,7 @@ import org.springframework.util.StringUtils;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
+import java.util.StringJoiner;
 import java.util.stream.Collectors;
 
 @Component
@@ -40,9 +41,17 @@ public class DeviceAlarmModelParser implements RuleModelParserStrategy {
         DeviceAlarmRule alarmRule = rule.getAlarmRule();
         alarmRule.validate();
 
+        RuleNodeModel conditionNode = new RuleNodeModel();
+        conditionNode.setId("conditions");
+        conditionNode.setName("预警条件");
+        conditionNode.setExecutor("device_alarm");
+        conditionNode.setConfiguration(Collections.singletonMap("rule", rule.getAlarmRule()));
+
         //处理定时触发
         {
-            List<DeviceAlarmRule.Trigger> timerTriggers = alarmRule.getTriggers().stream()
+            List<DeviceAlarmRule.Trigger> timerTriggers = alarmRule
+                .getTriggers()
+                .stream()
                 .filter(trigger -> trigger.getTrigger() == DeviceAlarmRule.TriggerType.timer)
                 .collect(Collectors.toList());
             int index = 0;
@@ -51,24 +60,27 @@ public class DeviceAlarmModelParser implements RuleModelParserStrategy {
                 if (msg == null) {
                     throw new UnsupportedOperationException("不支持定时条件类型:" + timerTrigger.getType());
                 }
+                //定时节点
                 RuleNodeModel timer = new RuleNodeModel();
                 timer.setId("timer:" + (++index));
                 timer.setName("定时发送设备消息");
                 timer.setExecutor("timer");
                 timer.setConfiguration(Collections.singletonMap("cron", timerTrigger.getCron()));
 
-                DeviceMessageSendTaskExecutorProvider.DeviceMessageSendConfig senderConfig = new DeviceMessageSendTaskExecutorProvider.DeviceMessageSendConfig();
-                senderConfig.setAsync(true);
-                senderConfig.setDeviceId(alarmRule.getDeviceId());
-                senderConfig.setProductId(alarmRule.getProductId());
-                senderConfig.setMessage(msg.toJson());
+                //发送指令节点
+                DeviceMessageSendTaskExecutorProvider.DeviceMessageSendConfig senderDeviceMessageSendConfig = new DeviceMessageSendTaskExecutorProvider.DeviceMessageSendConfig();
+                //同步等待回复
+                senderDeviceMessageSendConfig.setAsync(false);
+                senderDeviceMessageSendConfig.setStateOperator("direct");
+                senderDeviceMessageSendConfig.setDeviceId(alarmRule.getDeviceId());
+                senderDeviceMessageSendConfig.setProductId(alarmRule.getProductId());
+                senderDeviceMessageSendConfig.setMessage(msg.toJson());
 
                 RuleNodeModel messageSender = new RuleNodeModel();
                 messageSender.setId("message-sender:" + (++index));
                 messageSender.setName("定时发送设备消息");
                 messageSender.setExecutor("device-message-sender");
-                messageSender.setConfiguration(senderConfig.toMap());
-
+                messageSender.setConfiguration(senderDeviceMessageSendConfig.toMap());
                 RuleLink link = new RuleLink();
                 link.setId(timer.getId().concat(":").concat(messageSender.getId()));
                 link.setName("执行动作:" + index);
@@ -78,14 +90,18 @@ public class DeviceAlarmModelParser implements RuleModelParserStrategy {
                 messageSender.getInputs().add(link);
                 model.getNodes().add(timer);
                 model.getNodes().add(messageSender);
+
+                //将输出传递到告警节点
+                RuleLink toAlarm = new RuleLink();
+                toAlarm.setId(messageSender.getId().concat(":").concat(conditionNode.getId()));
+                toAlarm.setName("定时触发告警:" + index);
+                toAlarm.setSource(messageSender);
+                toAlarm.setTarget(conditionNode);
+                messageSender.getOutputs().add(toAlarm);
+                conditionNode.getInputs().add(toAlarm);
             }
         }
 
-        RuleNodeModel conditionNode = new RuleNodeModel();
-        conditionNode.setId("conditions");
-        conditionNode.setName("预警条件");
-        conditionNode.setExecutor("device_alarm");
-        conditionNode.setConfiguration(Collections.singletonMap("rule", rule.getAlarmRule()));
         model.getNodes().add(conditionNode);
         if (CollectionUtils.isNotEmpty(rule.getAlarmRule().getActions())) {
             int index = 0;