Skip to content

Commit e3a6189

Browse files
authored
Fix ticket event missing bug (#82)
* Fix ticket event missing bug * notes
1 parent 157bb1e commit e3a6189

File tree

3 files changed

+9
-10
lines changed

3 files changed

+9
-10
lines changed

analyzer-alert/src/main/java/cloud/erda/analyzer/alert/Main.java

+4-6
Original file line numberDiff line numberDiff line change
@@ -161,17 +161,12 @@ public static void main(String[] args) throws Exception {
161161
.setParallelism(parameterTool.getInt(STREAM_PARALLELISM_OPERATOR))
162162
.name("broadcast alert notify");
163163

164-
DataStream<AlertEvent> levelMatchedAlertEventsWithNotify = alertEventsWithNotify
165-
.filter(new AlertEventLevelFilterFunction())
166-
.setParallelism(parameterTool.getInt(STREAM_PARALLELISM_OPERATOR))
167-
.name("filter event notify level");
168-
169164
DataStream<NotifyEvent> notifyEventWithTemplate = notifyEventDataStream.connect(allUniversalTemplates.broadcast(StateDescriptors.notifyTemplate))
170165
.process(new NotifyTemplateProcessFunction(parameterTool.getLong(METRIC_METADATA_TTL, 7500), StateDescriptors.notifyTemplate))
171166
.setParallelism(parameterTool.getInt(STREAM_PARALLELISM_OPERATOR))
172167
.name("notify event with template");
173168

174-
DataStream<AlertEvent> alertEventsWithTemplate = levelMatchedAlertEventsWithNotify
169+
DataStream<AlertEvent> alertEventsWithTemplate = alertEventsWithNotify
175170
.connect(alertNotifyTemplateQuery.union(alertNotifyCustomTemplateQuery).broadcast(StateDescriptors.alertNotifyTemplateStateDescriptor))
176171
.process(new AlertNotifyTemplateBroadcastProcessFunction(parameterTool.getLong(METRIC_METADATA_TTL,
177172
75000), StateDescriptors.alertNotifyTemplateStateDescriptor))
@@ -243,6 +238,9 @@ public static void main(String[] args) throws Exception {
243238
}
244239

245240
DataStream<AlertEvent> alertEventLevel = alertEventsWithTemplate
241+
.filter(new AlertEventLevelFilterFunction())
242+
.setParallelism(parameterTool.getInt(STREAM_PARALLELISM_OPERATOR))
243+
.name("filter event notify level")
246244
.assignTimestampsAndWatermarks(new AlertEventWatermarkExtractor())
247245
.keyBy(new AlertEventGroupFunction())
248246
.window(TumblingProcessingTimeWindows.of(Time.seconds(15)))

analyzer-alert/src/main/java/cloud/erda/analyzer/alert/functions/AlertEventLevelFilterFunction.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -14,9 +14,9 @@ public boolean filter(AlertEvent alertEvent) throws Exception {
1414
AlertLevel eventLevel = alertEvent.getLevel();
1515
AlertLevel[] notifyLevels = alertEvent.getAlertNotify().getNotifyTarget().getLevels();
1616

17-
// compatible only when the get level fails (AlertLevel.UNKNOWN) and the notifyGroup's level is empty
18-
if (AlertLevel.UNKNOWN.equals(eventLevel)) {
19-
return notifyLevels.length == 0;
17+
// compatible only when the get level fails (AlertLevel.UNKNOWN) or the notifyGroup's level is empty
18+
if (AlertLevel.UNKNOWN.equals(eventLevel) || notifyLevels.length == 0) {
19+
return true;
2020
}
2121

2222
for (AlertLevel notifyLevel : notifyLevels) {

erda.yml

+2-1
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,8 @@ envs:
2121
TASK_MANAGER_MEMORY_MANAGED_FRACTION: "0.3"
2222
TASK_MANAGER_MEMORY_JVM_METASPACE_SIZE: "256M"
2323
TASK_MANAGER_JVM_EXIT_OOM: "true"
24-
FLINK_ENV_JAVA_OPTS_TM: "-XX:+UseG1GC -XX:+TraceClassLoading -XX:+TraceClassUnloading -XX:+PrintGC -XX:+PrintGCDetails"
24+
# FLINK_ENV_JAVA_OPTS_TM: "-XX:+UseG1GC -XX:+TraceClassLoading -XX:+TraceClassUnloading -XX:+PrintGC -XX:+PrintGCDetails"
25+
FLINK_ENV_JAVA_OPTS_TM: "-XX:+UseG1GC -XX:+PrintGC -XX:+PrintGCDetails"
2526
MONITOR_ADDR: "monitor:7096"
2627
EVENTBOX_ADDR: "eventbox:9528"
2728
ANALYZER_METRICS_ADDR: "analyzer-metrics:8081"

0 commit comments

Comments
 (0)