Wenjun Ruan
6 months ago
committed by
GitHub
99 changed files with 2336 additions and 882 deletions
@ -0,0 +1,100 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
|
||||
import java.util.List; |
||||
import java.util.concurrent.atomic.AtomicBoolean; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public abstract class AbstractEventFetcher<T> extends BaseDaemonThread implements EventFetcher<T> { |
||||
|
||||
protected static final int FETCH_SIZE = 100; |
||||
|
||||
protected static final long FETCH_INTERVAL = 5_000; |
||||
|
||||
protected final AlertHAServer alertHAServer; |
||||
|
||||
private final EventPendingQueue<T> eventPendingQueue; |
||||
|
||||
private final AtomicBoolean runningFlag = new AtomicBoolean(false); |
||||
|
||||
private Integer eventOffset; |
||||
|
||||
protected AbstractEventFetcher(String fetcherName, |
||||
AlertHAServer alertHAServer, |
||||
EventPendingQueue<T> eventPendingQueue) { |
||||
super(fetcherName); |
||||
this.alertHAServer = alertHAServer; |
||||
this.eventPendingQueue = eventPendingQueue; |
||||
this.eventOffset = -1; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
if (!runningFlag.compareAndSet(false, true)) { |
||||
throw new IllegalArgumentException("AlertEventFetcher is already started"); |
||||
} |
||||
log.info("AlertEventFetcher starting..."); |
||||
super.start(); |
||||
log.info("AlertEventFetcher started..."); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
while (runningFlag.get()) { |
||||
try { |
||||
if (!alertHAServer.isActive()) { |
||||
log.debug("The current node is not active, will not loop Alert"); |
||||
Thread.sleep(FETCH_INTERVAL); |
||||
continue; |
||||
} |
||||
List<T> pendingEvents = fetchPendingEvent(eventOffset); |
||||
if (CollectionUtils.isEmpty(pendingEvents)) { |
||||
log.debug("No pending events found"); |
||||
Thread.sleep(FETCH_INTERVAL); |
||||
continue; |
||||
} |
||||
for (T alert : pendingEvents) { |
||||
eventPendingQueue.put(alert); |
||||
} |
||||
eventOffset = Math.max(eventOffset, |
||||
pendingEvents.stream().map(this::getEventOffset).max(Integer::compareTo).get()); |
||||
} catch (InterruptedException interruptedException) { |
||||
Thread.currentThread().interrupt(); |
||||
} catch (Exception ex) { |
||||
log.error("AlertEventFetcher error", ex); |
||||
} |
||||
} |
||||
} |
||||
|
||||
protected abstract int getEventOffset(T event); |
||||
|
||||
@Override |
||||
public void shutdown() { |
||||
if (!runningFlag.compareAndSet(true, false)) { |
||||
log.warn("The AlertEventFetcher is not started"); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,101 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
import java.util.concurrent.ThreadPoolExecutor; |
||||
import java.util.concurrent.atomic.AtomicBoolean; |
||||
import java.util.concurrent.atomic.AtomicInteger; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public abstract class AbstractEventLoop<T> extends BaseDaemonThread implements EventLoop<T> { |
||||
|
||||
private final EventPendingQueue<T> eventPendingQueue; |
||||
|
||||
private final AtomicInteger handlingEventCount; |
||||
|
||||
private final int eventHandleWorkerNum; |
||||
|
||||
private final ThreadPoolExecutor threadPoolExecutor; |
||||
|
||||
private final AtomicBoolean runningFlag = new AtomicBoolean(false); |
||||
|
||||
protected AbstractEventLoop(String name, |
||||
ThreadPoolExecutor threadPoolExecutor, |
||||
EventPendingQueue<T> eventPendingQueue) { |
||||
super(name); |
||||
this.handlingEventCount = new AtomicInteger(0); |
||||
this.eventHandleWorkerNum = threadPoolExecutor.getMaximumPoolSize(); |
||||
this.threadPoolExecutor = threadPoolExecutor; |
||||
this.eventPendingQueue = eventPendingQueue; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized void start() { |
||||
if (!runningFlag.compareAndSet(false, true)) { |
||||
throw new IllegalArgumentException(getClass().getName() + " is already started"); |
||||
} |
||||
log.info("{} starting...", getClass().getName()); |
||||
super.start(); |
||||
log.info("{} started...", getClass().getName()); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
while (runningFlag.get()) { |
||||
try { |
||||
if (handlingEventCount.get() >= eventHandleWorkerNum) { |
||||
log.debug("There is no idle event worker, waiting for a while..."); |
||||
Thread.sleep(1000); |
||||
continue; |
||||
} |
||||
T pendingEvent = eventPendingQueue.take(); |
||||
handlingEventCount.incrementAndGet(); |
||||
CompletableFuture.runAsync(() -> handleEvent(pendingEvent), threadPoolExecutor) |
||||
.whenComplete((aVoid, throwable) -> { |
||||
if (throwable != null) { |
||||
log.error("Handle event: {} error", pendingEvent, throwable); |
||||
} |
||||
handlingEventCount.decrementAndGet(); |
||||
}); |
||||
} catch (InterruptedException interruptedException) { |
||||
Thread.currentThread().interrupt(); |
||||
log.error("Loop event thread has been interrupted..."); |
||||
break; |
||||
} catch (Exception ex) { |
||||
log.error("Loop event error", ex); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public int getHandlingEventCount() { |
||||
return handlingEventCount.get(); |
||||
} |
||||
|
||||
@Override |
||||
public void shutdown() { |
||||
if (!runningFlag.compareAndSet(true, false)) { |
||||
log.warn(getClass().getName() + " is not started"); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,53 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import java.util.concurrent.LinkedBlockingQueue; |
||||
|
||||
public abstract class AbstractEventPendingQueue<T> implements EventPendingQueue<T> { |
||||
|
||||
private final LinkedBlockingQueue<T> pendingAlertQueue; |
||||
|
||||
private final int capacity; |
||||
|
||||
protected AbstractEventPendingQueue(int capacity) { |
||||
this.capacity = capacity; |
||||
this.pendingAlertQueue = new LinkedBlockingQueue<>(capacity); |
||||
} |
||||
|
||||
@Override |
||||
public void put(T alert) throws InterruptedException { |
||||
pendingAlertQueue.put(alert); |
||||
} |
||||
|
||||
@Override |
||||
public T take() throws InterruptedException { |
||||
return pendingAlertQueue.take(); |
||||
} |
||||
|
||||
@Override |
||||
public int size() { |
||||
return pendingAlertQueue.size(); |
||||
} |
||||
|
||||
@Override |
||||
public int capacity() { |
||||
return capacity; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,191 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.alert.api.AlertChannel; |
||||
import org.apache.dolphinscheduler.alert.api.AlertConstants; |
||||
import org.apache.dolphinscheduler.alert.api.AlertData; |
||||
import org.apache.dolphinscheduler.alert.api.AlertInfo; |
||||
import org.apache.dolphinscheduler.alert.api.AlertResult; |
||||
import org.apache.dolphinscheduler.alert.plugin.AlertPluginManager; |
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.common.enums.AlertType; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertPluginInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertSendStatus; |
||||
import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse; |
||||
import org.apache.dolphinscheduler.spi.params.PluginParamsTransfer; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
import org.apache.commons.lang3.exception.ExceptionUtils; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Date; |
||||
import java.util.List; |
||||
import java.util.Optional; |
||||
import java.util.concurrent.CompletableFuture; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import com.google.common.base.Preconditions; |
||||
import com.google.common.collect.Lists; |
||||
|
||||
@Slf4j |
||||
public abstract class AbstractEventSender<T> implements EventSender<T> { |
||||
|
||||
protected final AlertPluginManager alertPluginManager; |
||||
|
||||
private final long sendEventTimeout; |
||||
|
||||
protected AbstractEventSender(AlertPluginManager alertPluginManager, long sendEventTimeout) { |
||||
this.alertPluginManager = alertPluginManager; |
||||
this.sendEventTimeout = sendEventTimeout; |
||||
} |
||||
|
||||
@Override |
||||
public void sendEvent(T event) { |
||||
List<AlertPluginInstance> alertPluginInstanceList = getAlertPluginInstanceList(event); |
||||
if (CollectionUtils.isEmpty(alertPluginInstanceList)) { |
||||
onError(event, "No bind plugin instance found"); |
||||
return; |
||||
} |
||||
AlertData alertData = getAlertData(event); |
||||
List<AlertSendStatus> alertSendStatuses = new ArrayList<>(); |
||||
for (AlertPluginInstance instance : alertPluginInstanceList) { |
||||
AlertResult alertResult = doSendEvent(instance, alertData); |
||||
AlertStatus alertStatus = |
||||
alertResult.isSuccess() ? AlertStatus.EXECUTION_SUCCESS : AlertStatus.EXECUTION_FAILURE; |
||||
AlertSendStatus alertSendStatus = AlertSendStatus.builder() |
||||
.alertId(getEventId(event)) |
||||
.alertPluginInstanceId(instance.getId()) |
||||
.sendStatus(alertStatus) |
||||
.log(JSONUtils.toJsonString(alertResult)) |
||||
.createTime(new Date()) |
||||
.build(); |
||||
alertSendStatuses.add(alertSendStatus); |
||||
} |
||||
long failureCount = alertSendStatuses.stream() |
||||
.map(alertSendStatus -> alertSendStatus.getSendStatus() == AlertStatus.EXECUTION_FAILURE) |
||||
.count(); |
||||
long successCount = alertSendStatuses.stream() |
||||
.map(alertSendStatus -> alertSendStatus.getSendStatus() == AlertStatus.EXECUTION_SUCCESS) |
||||
.count(); |
||||
if (successCount == 0) { |
||||
onError(event, JSONUtils.toJsonString(alertSendStatuses)); |
||||
} else { |
||||
if (failureCount > 0) { |
||||
onPartialSuccess(event, JSONUtils.toJsonString(alertSendStatuses)); |
||||
} else { |
||||
onSuccess(event, JSONUtils.toJsonString(alertSendStatuses)); |
||||
} |
||||
} |
||||
} |
||||
|
||||
public abstract List<AlertPluginInstance> getAlertPluginInstanceList(T event); |
||||
|
||||
public abstract AlertData getAlertData(T event); |
||||
|
||||
public abstract Integer getEventId(T event); |
||||
|
||||
public abstract void onError(T event, String log); |
||||
|
||||
public abstract void onPartialSuccess(T event, String log); |
||||
|
||||
public abstract void onSuccess(T event, String log); |
||||
|
||||
@Override |
||||
public AlertResult doSendEvent(AlertPluginInstance instance, AlertData alertData) { |
||||
int pluginDefineId = instance.getPluginDefineId(); |
||||
Optional<AlertChannel> alertChannelOptional = alertPluginManager.getAlertChannel(pluginDefineId); |
||||
if (!alertChannelOptional.isPresent()) { |
||||
return AlertResult.fail("Cannot find the alertPlugin: " + pluginDefineId); |
||||
} |
||||
AlertChannel alertChannel = alertChannelOptional.get(); |
||||
|
||||
AlertInfo alertInfo = AlertInfo.builder() |
||||
.alertData(alertData) |
||||
.alertParams(PluginParamsTransfer.getPluginParamsMap(instance.getPluginInstanceParams())) |
||||
.alertPluginInstanceId(instance.getId()) |
||||
.build(); |
||||
try { |
||||
AlertResult alertResult; |
||||
if (sendEventTimeout <= 0) { |
||||
if (alertData.getAlertType() == AlertType.CLOSE_ALERT.getCode()) { |
||||
alertResult = alertChannel.closeAlert(alertInfo); |
||||
} else { |
||||
alertResult = alertChannel.process(alertInfo); |
||||
} |
||||
} else { |
||||
CompletableFuture<AlertResult> future; |
||||
if (alertData.getAlertType() == AlertType.CLOSE_ALERT.getCode()) { |
||||
future = CompletableFuture.supplyAsync(() -> alertChannel.closeAlert(alertInfo)); |
||||
} else { |
||||
future = CompletableFuture.supplyAsync(() -> alertChannel.process(alertInfo)); |
||||
} |
||||
alertResult = future.get(sendEventTimeout, TimeUnit.MILLISECONDS); |
||||
} |
||||
checkNotNull(alertResult, "AlertResult cannot be null"); |
||||
return alertResult; |
||||
} catch (InterruptedException interruptedException) { |
||||
Thread.currentThread().interrupt(); |
||||
return AlertResult.fail(ExceptionUtils.getMessage(interruptedException)); |
||||
} catch (Exception e) { |
||||
log.error("Send alert data {} failed", alertData, e); |
||||
return AlertResult.fail(ExceptionUtils.getMessage(e)); |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public AlertSendResponse syncTestSend(int pluginDefineId, String pluginInstanceParams) { |
||||
|
||||
Optional<AlertChannel> alertChannelOptional = alertPluginManager.getAlertChannel(pluginDefineId); |
||||
if (!alertChannelOptional.isPresent()) { |
||||
AlertSendResponse.AlertSendResponseResult alertSendResponseResult = |
||||
AlertSendResponse.AlertSendResponseResult.fail("Cannot find the alertPlugin: " + pluginDefineId); |
||||
return AlertSendResponse.fail(Lists.newArrayList(alertSendResponseResult)); |
||||
} |
||||
AlertData alertData = AlertData.builder() |
||||
.title(AlertConstants.TEST_TITLE) |
||||
.content(AlertConstants.TEST_CONTENT) |
||||
.build(); |
||||
|
||||
AlertInfo alertInfo = AlertInfo.builder() |
||||
.alertData(alertData) |
||||
.alertParams(PluginParamsTransfer.getPluginParamsMap(pluginInstanceParams)) |
||||
.build(); |
||||
|
||||
try { |
||||
AlertResult alertResult = alertChannelOptional.get().process(alertInfo); |
||||
Preconditions.checkNotNull(alertResult, "AlertResult cannot be null"); |
||||
if (alertResult.isSuccess()) { |
||||
return AlertSendResponse |
||||
.success(Lists.newArrayList(AlertSendResponse.AlertSendResponseResult.success())); |
||||
} |
||||
return AlertSendResponse.fail( |
||||
Lists.newArrayList(AlertSendResponse.AlertSendResponseResult.fail(alertResult.getMessage()))); |
||||
} catch (Exception e) { |
||||
log.error("Test send alert error", e); |
||||
return new AlertSendResponse(false, |
||||
Lists.newArrayList(AlertSendResponse.AlertSendResponseResult.fail(ExceptionUtils.getMessage(e)))); |
||||
} |
||||
|
||||
} |
||||
} |
@ -0,0 +1,51 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.dao.AlertDao; |
||||
import org.apache.dolphinscheduler.dao.entity.Alert; |
||||
|
||||
import java.util.List; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class AlertEventFetcher extends AbstractEventFetcher<Alert> { |
||||
|
||||
private final AlertDao alertDao; |
||||
|
||||
public AlertEventFetcher(AlertHAServer alertHAServer, |
||||
AlertDao alertDao, |
||||
AlertEventPendingQueue alertEventPendingQueue) { |
||||
super("AlertEventFetcher", alertHAServer, alertEventPendingQueue); |
||||
this.alertDao = alertDao; |
||||
} |
||||
|
||||
@Override |
||||
public List<Alert> fetchPendingEvent(int eventOffset) { |
||||
return alertDao.listPendingAlerts(eventOffset); |
||||
} |
||||
|
||||
@Override |
||||
protected int getEventOffset(Alert event) { |
||||
return event.getId(); |
||||
} |
||||
} |
@ -0,0 +1,46 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.metrics.AlertServerMetrics; |
||||
import org.apache.dolphinscheduler.dao.entity.Alert; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class AlertEventLoop extends AbstractEventLoop<Alert> { |
||||
|
||||
private final AlertSender alertSender; |
||||
|
||||
public AlertEventLoop(AlertEventPendingQueue alertEventPendingQueue, |
||||
AlertSenderThreadPoolFactory alertSenderThreadPoolFactory, |
||||
AlertSender alertSender) { |
||||
super("AlertEventLoop", alertSenderThreadPoolFactory.getThreadPool(), alertEventPendingQueue); |
||||
this.alertSender = alertSender; |
||||
AlertServerMetrics.registerPendingAlertGauge(this::getHandlingEventCount); |
||||
} |
||||
|
||||
@Override |
||||
public void handleEvent(Alert event) { |
||||
alertSender.sendEvent(event); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,33 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.alert.metrics.AlertServerMetrics; |
||||
import org.apache.dolphinscheduler.dao.entity.Alert; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class AlertEventPendingQueue extends AbstractEventPendingQueue<Alert> { |
||||
|
||||
public AlertEventPendingQueue(AlertConfig alertConfig) { |
||||
super(alertConfig.getSenderParallelism() * 3 + 1); |
||||
AlertServerMetrics.registerPendingAlertGauge(this::size); |
||||
} |
||||
} |
@ -0,0 +1,36 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.registry.api.Registry; |
||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; |
||||
import org.apache.dolphinscheduler.registry.api.ha.AbstractHAServer; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class AlertHAServer extends AbstractHAServer { |
||||
|
||||
public AlertHAServer(Registry registry) { |
||||
super(registry, RegistryNodeType.ALERT_LOCK.getRegistryPath()); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,131 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.api.AlertData; |
||||
import org.apache.dolphinscheduler.alert.api.AlertResult; |
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.alert.plugin.AlertPluginManager; |
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.dao.AlertDao; |
||||
import org.apache.dolphinscheduler.dao.entity.Alert; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertPluginInstance; |
||||
import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.List; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class AlertSender extends AbstractEventSender<Alert> { |
||||
|
||||
private final AlertDao alertDao; |
||||
|
||||
public AlertSender(AlertDao alertDao, |
||||
AlertPluginManager alertPluginManager, |
||||
AlertConfig alertConfig) { |
||||
super(alertPluginManager, alertConfig.getWaitTimeout()); |
||||
this.alertDao = alertDao; |
||||
} |
||||
|
||||
/** |
||||
* sync send alert handler |
||||
* |
||||
* @param alertGroupId alertGroupId |
||||
* @param title title |
||||
* @param content content |
||||
* @return AlertSendResponseCommand |
||||
*/ |
||||
public AlertSendResponse syncHandler(int alertGroupId, String title, String content) { |
||||
List<AlertPluginInstance> alertInstanceList = alertDao.listInstanceByAlertGroupId(alertGroupId); |
||||
AlertData alertData = AlertData.builder() |
||||
.content(content) |
||||
.title(title) |
||||
.build(); |
||||
|
||||
boolean sendResponseStatus = true; |
||||
List<AlertSendResponse.AlertSendResponseResult> sendResponseResults = new ArrayList<>(); |
||||
|
||||
if (CollectionUtils.isEmpty(alertInstanceList)) { |
||||
AlertSendResponse.AlertSendResponseResult alertSendResponseResult = |
||||
new AlertSendResponse.AlertSendResponseResult(); |
||||
String message = String.format("Alert GroupId %s send error : not found alert instance", alertGroupId); |
||||
alertSendResponseResult.setSuccess(false); |
||||
alertSendResponseResult.setMessage(message); |
||||
sendResponseResults.add(alertSendResponseResult); |
||||
log.error("Alert GroupId {} send error : not found alert instance", alertGroupId); |
||||
return new AlertSendResponse(false, sendResponseResults); |
||||
} |
||||
|
||||
for (AlertPluginInstance instance : alertInstanceList) { |
||||
AlertResult alertResult = doSendEvent(instance, alertData); |
||||
if (alertResult != null) { |
||||
AlertSendResponse.AlertSendResponseResult alertSendResponseResult = |
||||
new AlertSendResponse.AlertSendResponseResult( |
||||
alertResult.isSuccess(), |
||||
alertResult.getMessage()); |
||||
sendResponseStatus = sendResponseStatus && alertSendResponseResult.isSuccess(); |
||||
sendResponseResults.add(alertSendResponseResult); |
||||
} |
||||
} |
||||
|
||||
return new AlertSendResponse(sendResponseStatus, sendResponseResults); |
||||
} |
||||
|
||||
@Override |
||||
public List<AlertPluginInstance> getAlertPluginInstanceList(Alert event) { |
||||
return alertDao.listInstanceByAlertGroupId(event.getAlertGroupId()); |
||||
} |
||||
|
||||
@Override |
||||
public AlertData getAlertData(Alert event) { |
||||
return AlertData.builder() |
||||
.id(event.getId()) |
||||
.content(event.getContent()) |
||||
.log(event.getLog()) |
||||
.title(event.getTitle()) |
||||
.alertType(event.getAlertType().getCode()) |
||||
.build(); |
||||
} |
||||
|
||||
@Override |
||||
public Integer getEventId(Alert event) { |
||||
return event.getId(); |
||||
} |
||||
|
||||
@Override |
||||
public void onError(Alert event, String log) { |
||||
alertDao.updateAlert(AlertStatus.EXECUTION_FAILURE, log, event.getId()); |
||||
} |
||||
|
||||
@Override |
||||
public void onPartialSuccess(Alert event, String log) { |
||||
alertDao.updateAlert(AlertStatus.EXECUTION_PARTIAL_SUCCESS, log, event.getId()); |
||||
} |
||||
|
||||
@Override |
||||
public void onSuccess(Alert event, String log) { |
||||
alertDao.updateAlert(AlertStatus.EXECUTION_SUCCESS, log, event.getId()); |
||||
} |
||||
} |
@ -0,0 +1,41 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
|
||||
import java.util.concurrent.ThreadPoolExecutor; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class AlertSenderThreadPoolFactory { |
||||
|
||||
private final ThreadPoolExecutor threadPool; |
||||
|
||||
public AlertSenderThreadPoolFactory(AlertConfig alertConfig) { |
||||
this.threadPool = ThreadUtils.newDaemonFixedThreadExecutor("AlertSenderThread", |
||||
alertConfig.getSenderParallelism()); |
||||
} |
||||
|
||||
public ThreadPoolExecutor getThreadPool() { |
||||
return threadPool; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,34 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import java.util.List; |
||||
|
||||
/** |
||||
* The interface responsible for fetching events. |
||||
* |
||||
* @param <T> the type of event |
||||
*/ |
||||
public interface EventFetcher<T> { |
||||
|
||||
void start(); |
||||
|
||||
List<T> fetchPendingEvent(int eventOffset); |
||||
|
||||
void shutdown(); |
||||
} |
@ -0,0 +1,47 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
/** |
||||
* The interface responsible for consuming event from upstream, e.g {@link EventPendingQueue}. |
||||
* |
||||
* @param <T> the type of event |
||||
*/ |
||||
public interface EventLoop<T> { |
||||
|
||||
/** |
||||
* Start the event loop, once the event loop is started, it will keep consuming event from upstream. |
||||
*/ |
||||
void start(); |
||||
|
||||
/** |
||||
* Handle the given event. |
||||
*/ |
||||
void handleEvent(T event); |
||||
|
||||
/** |
||||
* Get the count of handling event. |
||||
*/ |
||||
int getHandlingEventCount(); |
||||
|
||||
/** |
||||
* Shutdown the event loop, once the event loop is shutdown, it will stop consuming event from upstream. |
||||
*/ |
||||
void shutdown(); |
||||
|
||||
} |
@ -0,0 +1,34 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
/** |
||||
* The interface responsible for managing pending events. |
||||
* |
||||
* @param <T> the type of event |
||||
*/ |
||||
public interface EventPendingQueue<T> { |
||||
|
||||
void put(T alert) throws InterruptedException; |
||||
|
||||
T take() throws InterruptedException; |
||||
|
||||
int size(); |
||||
|
||||
int capacity(); |
||||
} |
@ -0,0 +1,33 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.api.AlertData; |
||||
import org.apache.dolphinscheduler.alert.api.AlertResult; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertPluginInstance; |
||||
import org.apache.dolphinscheduler.extract.alert.request.AlertSendResponse; |
||||
|
||||
public interface EventSender<T> { |
||||
|
||||
void sendEvent(T event); |
||||
|
||||
AlertResult doSendEvent(AlertPluginInstance instance, AlertData alertData); |
||||
|
||||
AlertSendResponse syncTestSend(int pluginDefineId, String pluginInstanceParams); |
||||
|
||||
} |
@ -0,0 +1,51 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.repository.ListenerEventDao; |
||||
|
||||
import java.util.List; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class ListenerEventFetcher extends AbstractEventFetcher<ListenerEvent> { |
||||
|
||||
private final ListenerEventDao listenerEventDao; |
||||
|
||||
protected ListenerEventFetcher(AlertHAServer alertHAServer, |
||||
ListenerEventDao listenerEventDao, |
||||
ListenerEventPendingQueue listenerEventPendingQueue) { |
||||
super("ListenerEventFetcher", alertHAServer, listenerEventPendingQueue); |
||||
this.listenerEventDao = listenerEventDao; |
||||
} |
||||
|
||||
@Override |
||||
protected int getEventOffset(ListenerEvent event) { |
||||
return event.getId(); |
||||
} |
||||
|
||||
@Override |
||||
public List<ListenerEvent> fetchPendingEvent(int eventOffset) { |
||||
return listenerEventDao.listingPendingEvents(eventOffset, FETCH_SIZE); |
||||
} |
||||
} |
@ -0,0 +1,40 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class ListenerEventLoop extends AbstractEventLoop<ListenerEvent> { |
||||
|
||||
private final ListenerEventSender listenerEventSender; |
||||
|
||||
protected ListenerEventLoop(AlertSenderThreadPoolFactory alertSenderThreadPoolFactory, |
||||
ListenerEventSender listenerEventSender, |
||||
ListenerEventPendingQueue listenerEventPendingQueue) { |
||||
super("ListenerEventLoop", alertSenderThreadPoolFactory.getThreadPool(), listenerEventPendingQueue); |
||||
this.listenerEventSender = listenerEventSender; |
||||
} |
||||
|
||||
@Override |
||||
public void handleEvent(ListenerEvent event) { |
||||
listenerEventSender.sendEvent(event); |
||||
} |
||||
} |
@ -0,0 +1,32 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class ListenerEventPendingQueue extends AbstractEventPendingQueue<ListenerEvent> { |
||||
|
||||
public ListenerEventPendingQueue(AlertConfig alertConfig) { |
||||
super(alertConfig.getSenderParallelism() * 3 + 1); |
||||
} |
||||
|
||||
} |
@ -1,262 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.api.AlertChannel; |
||||
import org.apache.dolphinscheduler.alert.api.AlertData; |
||||
import org.apache.dolphinscheduler.alert.api.AlertInfo; |
||||
import org.apache.dolphinscheduler.alert.api.AlertResult; |
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.alert.plugin.AlertPluginManager; |
||||
import org.apache.dolphinscheduler.common.constants.Constants; |
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.common.enums.AlertType; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager; |
||||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; |
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertPluginInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertSendStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.AbstractListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionCreatedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionDeletedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionUpdatedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessEndListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessFailListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessStartListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ServerDownListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskEndListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskFailListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskStartListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.mapper.AlertPluginInstanceMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.ListenerEventMapper; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
import org.apache.curator.shaded.com.google.common.collect.Lists; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Date; |
||||
import java.util.List; |
||||
import java.util.Map; |
||||
import java.util.Optional; |
||||
import java.util.concurrent.CompletableFuture; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import javax.annotation.Nullable; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.beans.factory.annotation.Value; |
||||
import org.springframework.stereotype.Service; |
||||
|
||||
@Service |
||||
@Slf4j |
||||
public final class ListenerEventPostService extends BaseDaemonThread implements AutoCloseable { |
||||
|
||||
@Value("${alert.query_alert_threshold:100}") |
||||
private Integer QUERY_ALERT_THRESHOLD; |
||||
@Autowired |
||||
private ListenerEventMapper listenerEventMapper; |
||||
@Autowired |
||||
private AlertPluginInstanceMapper alertPluginInstanceMapper; |
||||
@Autowired |
||||
private AlertPluginManager alertPluginManager; |
||||
@Autowired |
||||
private AlertConfig alertConfig; |
||||
|
||||
public ListenerEventPostService() { |
||||
super("ListenerEventPostService"); |
||||
} |
||||
|
||||
@Override |
||||
public void run() { |
||||
log.info("listener event post thread started"); |
||||
while (!ServerLifeCycleManager.isStopped()) { |
||||
try { |
||||
List<ListenerEvent> listenerEvents = listenerEventMapper |
||||
.listingListenerEventByStatus(AlertStatus.WAIT_EXECUTION, QUERY_ALERT_THRESHOLD); |
||||
if (CollectionUtils.isEmpty(listenerEvents)) { |
||||
log.debug("There is no waiting listener events"); |
||||
continue; |
||||
} |
||||
this.send(listenerEvents); |
||||
} catch (Exception e) { |
||||
log.error("listener event post thread meet an exception", e); |
||||
} finally { |
||||
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS * 5L); |
||||
} |
||||
} |
||||
log.info("listener event post thread stopped"); |
||||
} |
||||
|
||||
public void send(List<ListenerEvent> listenerEvents) { |
||||
for (ListenerEvent listenerEvent : listenerEvents) { |
||||
int eventId = listenerEvent.getId(); |
||||
List<AlertPluginInstance> globalAlertInstanceList = |
||||
alertPluginInstanceMapper.queryAllGlobalAlertPluginInstanceList(); |
||||
if (CollectionUtils.isEmpty(globalAlertInstanceList)) { |
||||
log.error("post listener event fail,no bind global plugin instance."); |
||||
listenerEventMapper.updateListenerEvent(eventId, AlertStatus.EXECUTION_FAILURE, |
||||
"no bind plugin instance", new Date()); |
||||
continue; |
||||
} |
||||
AbstractListenerEvent event = generateEventFromContent(listenerEvent); |
||||
if (event == null) { |
||||
log.error("parse listener event to abstract listener event fail.ed {}", listenerEvent.getContent()); |
||||
listenerEventMapper.updateListenerEvent(eventId, AlertStatus.EXECUTION_FAILURE, |
||||
"parse listener event to abstract listener event failed", new Date()); |
||||
continue; |
||||
} |
||||
List<AbstractListenerEvent> events = Lists.newArrayList(event); |
||||
AlertData alertData = AlertData.builder() |
||||
.id(eventId) |
||||
.content(JSONUtils.toJsonString(events)) |
||||
.log(listenerEvent.getLog()) |
||||
.title(event.getTitle()) |
||||
.warnType(WarningType.GLOBAL.getCode()) |
||||
.alertType(event.getEventType().getCode()) |
||||
.build(); |
||||
|
||||
int sendSuccessCount = 0; |
||||
List<AlertSendStatus> failedPostResults = new ArrayList<>(); |
||||
for (AlertPluginInstance instance : globalAlertInstanceList) { |
||||
AlertResult alertResult = this.alertResultHandler(instance, alertData); |
||||
if (alertResult != null) { |
||||
AlertStatus sendStatus = Boolean.parseBoolean(alertResult.getStatus()) |
||||
? AlertStatus.EXECUTION_SUCCESS |
||||
: AlertStatus.EXECUTION_FAILURE; |
||||
if (AlertStatus.EXECUTION_SUCCESS.equals(sendStatus)) { |
||||
sendSuccessCount++; |
||||
} else { |
||||
AlertSendStatus alertSendStatus = AlertSendStatus.builder() |
||||
.alertId(eventId) |
||||
.alertPluginInstanceId(instance.getId()) |
||||
.sendStatus(sendStatus) |
||||
.log(JSONUtils.toJsonString(alertResult)) |
||||
.createTime(new Date()) |
||||
.build(); |
||||
failedPostResults.add(alertSendStatus); |
||||
} |
||||
} |
||||
} |
||||
if (sendSuccessCount == globalAlertInstanceList.size()) { |
||||
listenerEventMapper.deleteById(eventId); |
||||
} else { |
||||
AlertStatus alertStatus = |
||||
sendSuccessCount == 0 ? AlertStatus.EXECUTION_FAILURE : AlertStatus.EXECUTION_PARTIAL_SUCCESS; |
||||
listenerEventMapper.updateListenerEvent(eventId, alertStatus, JSONUtils.toJsonString(failedPostResults), |
||||
new Date()); |
||||
} |
||||
} |
||||
} |
||||
|
||||
/** |
||||
* alert result handler |
||||
* |
||||
* @param instance instance |
||||
* @param alertData alertData |
||||
* @return AlertResult |
||||
*/ |
||||
private @Nullable AlertResult alertResultHandler(AlertPluginInstance instance, AlertData alertData) { |
||||
String pluginInstanceName = instance.getInstanceName(); |
||||
int pluginDefineId = instance.getPluginDefineId(); |
||||
Optional<AlertChannel> alertChannelOptional = alertPluginManager.getAlertChannel(instance.getPluginDefineId()); |
||||
if (!alertChannelOptional.isPresent()) { |
||||
String message = |
||||
String.format("Global Alert Plugin %s send error: the channel doesn't exist, pluginDefineId: %s", |
||||
pluginInstanceName, |
||||
pluginDefineId); |
||||
log.error("Global Alert Plugin {} send error : not found plugin {}", pluginInstanceName, pluginDefineId); |
||||
return new AlertResult("false", message); |
||||
} |
||||
AlertChannel alertChannel = alertChannelOptional.get(); |
||||
|
||||
Map<String, String> paramsMap = JSONUtils.toMap(instance.getPluginInstanceParams()); |
||||
|
||||
AlertInfo alertInfo = AlertInfo.builder() |
||||
.alertData(alertData) |
||||
.alertParams(paramsMap) |
||||
.alertPluginInstanceId(instance.getId()) |
||||
.build(); |
||||
int waitTimeout = alertConfig.getWaitTimeout(); |
||||
try { |
||||
AlertResult alertResult; |
||||
if (waitTimeout <= 0) { |
||||
if (alertData.getAlertType() == AlertType.CLOSE_ALERT.getCode()) { |
||||
alertResult = alertChannel.closeAlert(alertInfo); |
||||
} else { |
||||
alertResult = alertChannel.process(alertInfo); |
||||
} |
||||
} else { |
||||
CompletableFuture<AlertResult> future; |
||||
if (alertData.getAlertType() == AlertType.CLOSE_ALERT.getCode()) { |
||||
future = CompletableFuture.supplyAsync(() -> alertChannel.closeAlert(alertInfo)); |
||||
} else { |
||||
future = CompletableFuture.supplyAsync(() -> alertChannel.process(alertInfo)); |
||||
} |
||||
alertResult = future.get(waitTimeout, TimeUnit.MILLISECONDS); |
||||
} |
||||
if (alertResult == null) { |
||||
throw new RuntimeException("Alert result cannot be null"); |
||||
} |
||||
return alertResult; |
||||
} catch (InterruptedException e) { |
||||
log.error("post listener event error alert data id :{},", alertData.getId(), e); |
||||
Thread.currentThread().interrupt(); |
||||
return new AlertResult("false", e.getMessage()); |
||||
} catch (Exception e) { |
||||
log.error("post listener event error alert data id :{},", alertData.getId(), e); |
||||
return new AlertResult("false", e.getMessage()); |
||||
} |
||||
} |
||||
|
||||
private AbstractListenerEvent generateEventFromContent(ListenerEvent listenerEvent) { |
||||
String content = listenerEvent.getContent(); |
||||
switch (listenerEvent.getEventType()) { |
||||
case SERVER_DOWN: |
||||
return JSONUtils.parseObject(content, ServerDownListenerEvent.class); |
||||
case PROCESS_DEFINITION_CREATED: |
||||
return JSONUtils.parseObject(content, ProcessDefinitionCreatedListenerEvent.class); |
||||
case PROCESS_DEFINITION_UPDATED: |
||||
return JSONUtils.parseObject(content, ProcessDefinitionUpdatedListenerEvent.class); |
||||
case PROCESS_DEFINITION_DELETED: |
||||
return JSONUtils.parseObject(content, ProcessDefinitionDeletedListenerEvent.class); |
||||
case PROCESS_START: |
||||
return JSONUtils.parseObject(content, ProcessStartListenerEvent.class); |
||||
case PROCESS_END: |
||||
return JSONUtils.parseObject(content, ProcessEndListenerEvent.class); |
||||
case PROCESS_FAIL: |
||||
return JSONUtils.parseObject(content, ProcessFailListenerEvent.class); |
||||
case TASK_START: |
||||
return JSONUtils.parseObject(content, TaskStartListenerEvent.class); |
||||
case TASK_END: |
||||
return JSONUtils.parseObject(content, TaskEndListenerEvent.class); |
||||
case TASK_FAIL: |
||||
return JSONUtils.parseObject(content, TaskFailListenerEvent.class); |
||||
default: |
||||
return null; |
||||
} |
||||
} |
||||
@Override |
||||
public void close() { |
||||
log.info("Closed ListenerEventPostService..."); |
||||
} |
||||
} |
@ -0,0 +1,146 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import org.apache.dolphinscheduler.alert.api.AlertData; |
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.alert.plugin.AlertPluginManager; |
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.AlertPluginInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.AbstractListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionCreatedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionDeletedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessDefinitionUpdatedListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessEndListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessFailListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ProcessStartListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.ServerDownListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskEndListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskFailListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.entity.event.TaskStartListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.mapper.AlertPluginInstanceMapper; |
||||
import org.apache.dolphinscheduler.dao.repository.ListenerEventDao; |
||||
|
||||
import org.apache.curator.shaded.com.google.common.collect.Lists; |
||||
|
||||
import java.util.Date; |
||||
import java.util.List; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class ListenerEventSender extends AbstractEventSender<ListenerEvent> { |
||||
|
||||
private final ListenerEventDao listenerEventDao; |
||||
|
||||
private final AlertPluginInstanceMapper alertPluginInstanceMapper; |
||||
|
||||
public ListenerEventSender(ListenerEventDao listenerEventDao, |
||||
AlertPluginInstanceMapper alertPluginInstanceMapper, |
||||
AlertPluginManager alertPluginManager, |
||||
AlertConfig alertConfig) { |
||||
super(alertPluginManager, alertConfig.getWaitTimeout()); |
||||
this.listenerEventDao = listenerEventDao; |
||||
this.alertPluginInstanceMapper = alertPluginInstanceMapper; |
||||
} |
||||
|
||||
private AbstractListenerEvent generateEventFromContent(ListenerEvent listenerEvent) { |
||||
String content = listenerEvent.getContent(); |
||||
AbstractListenerEvent event = null; |
||||
switch (listenerEvent.getEventType()) { |
||||
case SERVER_DOWN: |
||||
event = JSONUtils.parseObject(content, ServerDownListenerEvent.class); |
||||
break; |
||||
case PROCESS_DEFINITION_CREATED: |
||||
event = JSONUtils.parseObject(content, ProcessDefinitionCreatedListenerEvent.class); |
||||
break; |
||||
case PROCESS_DEFINITION_UPDATED: |
||||
event = JSONUtils.parseObject(content, ProcessDefinitionUpdatedListenerEvent.class); |
||||
break; |
||||
case PROCESS_DEFINITION_DELETED: |
||||
event = JSONUtils.parseObject(content, ProcessDefinitionDeletedListenerEvent.class); |
||||
break; |
||||
case PROCESS_START: |
||||
event = JSONUtils.parseObject(content, ProcessStartListenerEvent.class); |
||||
break; |
||||
case PROCESS_END: |
||||
event = JSONUtils.parseObject(content, ProcessEndListenerEvent.class); |
||||
break; |
||||
case PROCESS_FAIL: |
||||
event = JSONUtils.parseObject(content, ProcessFailListenerEvent.class); |
||||
break; |
||||
case TASK_START: |
||||
event = JSONUtils.parseObject(content, TaskStartListenerEvent.class); |
||||
break; |
||||
case TASK_END: |
||||
event = JSONUtils.parseObject(content, TaskEndListenerEvent.class); |
||||
break; |
||||
case TASK_FAIL: |
||||
event = JSONUtils.parseObject(content, TaskFailListenerEvent.class); |
||||
break; |
||||
default: |
||||
throw new IllegalArgumentException("Unsupported event type: " + listenerEvent.getEventType()); |
||||
} |
||||
if (event == null) { |
||||
throw new IllegalArgumentException("Failed to parse event from content: " + content); |
||||
} |
||||
return event; |
||||
} |
||||
|
||||
@Override |
||||
public List<AlertPluginInstance> getAlertPluginInstanceList(ListenerEvent event) { |
||||
return alertPluginInstanceMapper.queryAllGlobalAlertPluginInstanceList(); |
||||
} |
||||
|
||||
@Override |
||||
public AlertData getAlertData(ListenerEvent listenerEvent) { |
||||
AbstractListenerEvent event = generateEventFromContent(listenerEvent); |
||||
return AlertData.builder() |
||||
.id(listenerEvent.getId()) |
||||
.content(JSONUtils.toJsonString(Lists.newArrayList(event))) |
||||
.log(listenerEvent.getLog()) |
||||
.title(event.getTitle()) |
||||
.alertType(event.getEventType().getCode()) |
||||
.build(); |
||||
} |
||||
|
||||
@Override |
||||
public Integer getEventId(ListenerEvent event) { |
||||
return event.getId(); |
||||
} |
||||
|
||||
@Override |
||||
public void onError(ListenerEvent event, String log) { |
||||
listenerEventDao.updateListenerEvent(event.getId(), AlertStatus.EXECUTION_FAILURE, log, new Date()); |
||||
} |
||||
|
||||
@Override |
||||
public void onPartialSuccess(ListenerEvent event, String log) { |
||||
listenerEventDao.updateListenerEvent(event.getId(), AlertStatus.EXECUTION_PARTIAL_SUCCESS, log, new Date()); |
||||
} |
||||
|
||||
@Override |
||||
public void onSuccess(ListenerEvent event, String log) { |
||||
listenerEventDao.updateListenerEvent(event.getId(), AlertStatus.EXECUTION_FAILURE, log, new Date()); |
||||
} |
||||
} |
@ -0,0 +1,43 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.config; |
||||
|
||||
import static com.google.common.truth.Truth.assertThat; |
||||
|
||||
import java.time.Duration; |
||||
|
||||
import org.junit.jupiter.api.Test; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.boot.test.autoconfigure.web.servlet.AutoConfigureMockMvc; |
||||
import org.springframework.boot.test.context.SpringBootTest; |
||||
|
||||
@AutoConfigureMockMvc |
||||
@SpringBootTest(classes = AlertConfig.class) |
||||
class AlertConfigTest { |
||||
|
||||
@Autowired |
||||
private AlertConfig alertConfig; |
||||
|
||||
@Test |
||||
void testValidate() { |
||||
assertThat(alertConfig.getWaitTimeout()).isEqualTo(10); |
||||
assertThat(alertConfig.getMaxHeartbeatInterval()).isEqualTo(Duration.ofSeconds(59)); |
||||
assertThat(alertConfig.getSenderParallelism()).isEqualTo(101); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,94 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import static com.google.common.truth.Truth.assertThat; |
||||
import static org.awaitility.Awaitility.await; |
||||
import static org.junit.jupiter.api.Assertions.assertThrowsExactly; |
||||
|
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
import org.apache.dolphinscheduler.dao.entity.Alert; |
||||
|
||||
import java.time.Duration; |
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import lombok.SneakyThrows; |
||||
|
||||
import org.awaitility.core.ConditionTimeoutException; |
||||
import org.junit.jupiter.api.BeforeEach; |
||||
import org.junit.jupiter.api.Test; |
||||
|
||||
class AlertEventPendingQueueTest { |
||||
|
||||
private AlertEventPendingQueue alertEventPendingQueue; |
||||
|
||||
private static final int QUEUE_SIZE = 10; |
||||
|
||||
@BeforeEach |
||||
public void before() { |
||||
AlertConfig alertConfig = new AlertConfig(); |
||||
alertConfig.setSenderParallelism(QUEUE_SIZE); |
||||
this.alertEventPendingQueue = new AlertEventPendingQueue(alertConfig); |
||||
} |
||||
|
||||
@SneakyThrows |
||||
@Test |
||||
void put() { |
||||
for (int i = 0; i < alertEventPendingQueue.capacity(); i++) { |
||||
alertEventPendingQueue.put(new Alert()); |
||||
} |
||||
|
||||
CompletableFuture<Void> completableFuture = CompletableFuture.runAsync(() -> { |
||||
try { |
||||
alertEventPendingQueue.put(new Alert()); |
||||
System.out.println(alertEventPendingQueue.size()); |
||||
} catch (InterruptedException e) { |
||||
throw new RuntimeException(e); |
||||
} |
||||
}); |
||||
assertThrowsExactly(ConditionTimeoutException.class, |
||||
() -> await() |
||||
.timeout(Duration.ofSeconds(2)) |
||||
.until(completableFuture::isDone)); |
||||
|
||||
} |
||||
|
||||
@Test |
||||
void take() { |
||||
CompletableFuture<Void> completableFuture = CompletableFuture.runAsync(() -> { |
||||
try { |
||||
alertEventPendingQueue.take(); |
||||
} catch (InterruptedException e) { |
||||
throw new RuntimeException(e); |
||||
} |
||||
}); |
||||
assertThrowsExactly(ConditionTimeoutException.class, |
||||
() -> await() |
||||
.timeout(Duration.ofSeconds(2)) |
||||
.until(completableFuture::isDone)); |
||||
} |
||||
|
||||
@SneakyThrows |
||||
@Test |
||||
void size() { |
||||
for (int i = 0; i < alertEventPendingQueue.capacity(); i++) { |
||||
alertEventPendingQueue.put(new Alert()); |
||||
assertThat(alertEventPendingQueue.size()).isEqualTo(i + 1); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,44 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.alert.service; |
||||
|
||||
import static com.google.common.truth.Truth.assertThat; |
||||
|
||||
import org.apache.dolphinscheduler.alert.config.AlertConfig; |
||||
|
||||
import java.util.concurrent.ThreadPoolExecutor; |
||||
|
||||
import org.junit.jupiter.api.Test; |
||||
import org.junit.jupiter.api.extension.ExtendWith; |
||||
import org.mockito.junit.jupiter.MockitoExtension; |
||||
|
||||
@ExtendWith(MockitoExtension.class) |
||||
class AlertSenderThreadPoolFactoryTest { |
||||
|
||||
private final AlertConfig alertConfig = new AlertConfig(); |
||||
|
||||
private final AlertSenderThreadPoolFactory alertSenderThreadPoolFactory = |
||||
new AlertSenderThreadPoolFactory(alertConfig); |
||||
|
||||
@Test |
||||
void getThreadPool() { |
||||
ThreadPoolExecutor threadPool = alertSenderThreadPoolFactory.getThreadPool(); |
||||
assertThat(threadPool.getCorePoolSize()).isEqualTo(alertConfig.getSenderParallelism()); |
||||
assertThat(threadPool.getMaximumPoolSize()).isEqualTo(alertConfig.getSenderParallelism()); |
||||
} |
||||
} |
@ -0,0 +1,107 @@
|
||||
# |
||||
# Licensed to the Apache Software Foundation (ASF) under one or more |
||||
# contributor license agreements. See the NOTICE file distributed with |
||||
# this work for additional information regarding copyright ownership. |
||||
# The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
# (the "License"); you may not use this file except in compliance with |
||||
# the License. You may obtain a copy of the License at |
||||
# |
||||
# http://www.apache.org/licenses/LICENSE-2.0 |
||||
# |
||||
# Unless required by applicable law or agreed to in writing, software |
||||
# distributed under the License is distributed on an "AS IS" BASIS, |
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
# See the License for the specific language governing permissions and |
||||
# limitations under the License. |
||||
# |
||||
|
||||
spring: |
||||
profiles: |
||||
active: postgresql |
||||
jackson: |
||||
time-zone: UTC |
||||
date-format: "yyyy-MM-dd HH:mm:ss" |
||||
banner: |
||||
charset: UTF-8 |
||||
datasource: |
||||
driver-class-name: org.postgresql.Driver |
||||
url: jdbc:postgresql://127.0.0.1:5432/dolphinscheduler |
||||
username: root |
||||
password: root |
||||
hikari: |
||||
connection-test-query: select 1 |
||||
pool-name: DolphinScheduler |
||||
|
||||
# Mybatis-plus configuration, you don't need to change it |
||||
mybatis-plus: |
||||
mapper-locations: classpath:org/apache/dolphinscheduler/dao/mapper/*Mapper.xml |
||||
type-aliases-package: org.apache.dolphinscheduler.dao.entity |
||||
configuration: |
||||
cache-enabled: false |
||||
call-setters-on-nulls: true |
||||
map-underscore-to-camel-case: true |
||||
jdbc-type-for-null: NULL |
||||
global-config: |
||||
db-config: |
||||
id-type: auto |
||||
banner: false |
||||
|
||||
server: |
||||
port: 50053 |
||||
|
||||
management: |
||||
endpoints: |
||||
web: |
||||
exposure: |
||||
include: health,metrics,prometheus |
||||
endpoint: |
||||
health: |
||||
enabled: true |
||||
show-details: always |
||||
health: |
||||
db: |
||||
enabled: true |
||||
defaults: |
||||
enabled: false |
||||
metrics: |
||||
tags: |
||||
application: ${spring.application.name} |
||||
|
||||
alert: |
||||
port: 50052 |
||||
# Mark each alert of alert server if late after x milliseconds as failed. |
||||
# Define value is (0 = infinite), and alert server would be waiting alert result. |
||||
wait-timeout: 10 |
||||
max-heartbeat-interval: 59s |
||||
# The maximum number of alerts that can be processed in parallel |
||||
sender-parallelism: 101 |
||||
|
||||
registry: |
||||
type: zookeeper |
||||
zookeeper: |
||||
namespace: dolphinscheduler |
||||
connect-string: localhost:2181 |
||||
retry-policy: |
||||
base-sleep-time: 60ms |
||||
max-sleep: 300ms |
||||
max-retries: 5 |
||||
session-timeout: 30s |
||||
connection-timeout: 9s |
||||
block-until-connected: 600ms |
||||
digest: ~ |
||||
|
||||
metrics: |
||||
enabled: true |
||||
|
||||
# Override by profile |
||||
|
||||
--- |
||||
spring: |
||||
config: |
||||
activate: |
||||
on-profile: mysql |
||||
datasource: |
||||
driver-class-name: com.mysql.cj.jdbc.Driver |
||||
url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler |
||||
username: root |
||||
password: root |
@ -0,0 +1,31 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.dao.repository; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
|
||||
import java.util.Date; |
||||
import java.util.List; |
||||
|
||||
public interface ListenerEventDao extends IDao<ListenerEvent> { |
||||
|
||||
List<ListenerEvent> listingPendingEvents(int minId, int limit); |
||||
|
||||
void updateListenerEvent(int eventId, AlertStatus alertStatus, String message, Date date); |
||||
} |
@ -0,0 +1,51 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.dao.repository.impl; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.mapper.ListenerEventMapper; |
||||
import org.apache.dolphinscheduler.dao.repository.BaseDao; |
||||
import org.apache.dolphinscheduler.dao.repository.ListenerEventDao; |
||||
|
||||
import java.util.Date; |
||||
import java.util.List; |
||||
|
||||
import lombok.NonNull; |
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Repository; |
||||
|
||||
@Slf4j |
||||
@Repository |
||||
public class ListenerEventDaoImpl extends BaseDao<ListenerEvent, ListenerEventMapper> implements ListenerEventDao { |
||||
|
||||
public ListenerEventDaoImpl(@NonNull ListenerEventMapper listenerEventMapper) { |
||||
super(listenerEventMapper); |
||||
} |
||||
|
||||
@Override |
||||
public List<ListenerEvent> listingPendingEvents(int minId, int limit) { |
||||
return mybatisMapper.listingListenerEventByStatus(minId, AlertStatus.WAIT_EXECUTION.getCode(), limit); |
||||
} |
||||
|
||||
@Override |
||||
public void updateListenerEvent(int eventId, AlertStatus alertStatus, String message, Date date) { |
||||
mybatisMapper.updateListenerEvent(eventId, alertStatus, message, date); |
||||
} |
||||
} |
@ -0,0 +1,79 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.dao.repository.impl; |
||||
|
||||
import static com.google.common.truth.Truth.assertThat; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.AlertStatus; |
||||
import org.apache.dolphinscheduler.common.enums.ListenerEventType; |
||||
import org.apache.dolphinscheduler.dao.BaseDaoTest; |
||||
import org.apache.dolphinscheduler.dao.entity.ListenerEvent; |
||||
import org.apache.dolphinscheduler.dao.repository.ListenerEventDao; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import org.junit.jupiter.api.Test; |
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
|
||||
class ListenerEventDaoImplTest extends BaseDaoTest { |
||||
|
||||
@Autowired |
||||
private ListenerEventDao listenerEventDao; |
||||
|
||||
@Test |
||||
void listingPendingEvents() { |
||||
int minId = -1; |
||||
int limit = 10; |
||||
assertThat(listenerEventDao.listingPendingEvents(minId, limit)).isEmpty(); |
||||
|
||||
ListenerEvent listenerEvent = ListenerEvent.builder() |
||||
.eventType(ListenerEventType.SERVER_DOWN) |
||||
.sign("test") |
||||
.createTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.postStatus(AlertStatus.WAIT_EXECUTION) |
||||
.build(); |
||||
listenerEventDao.insert(listenerEvent); |
||||
|
||||
listenerEvent = ListenerEvent.builder() |
||||
.eventType(ListenerEventType.SERVER_DOWN) |
||||
.sign("test") |
||||
.createTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.postStatus(AlertStatus.EXECUTION_SUCCESS) |
||||
.build(); |
||||
listenerEventDao.insert(listenerEvent); |
||||
|
||||
assertThat(listenerEventDao.listingPendingEvents(minId, limit)).hasSize(1); |
||||
} |
||||
|
||||
@Test |
||||
void updateListenerEvent() { |
||||
ListenerEvent listenerEvent = ListenerEvent.builder() |
||||
.eventType(ListenerEventType.SERVER_DOWN) |
||||
.sign("test") |
||||
.createTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.postStatus(AlertStatus.WAIT_EXECUTION) |
||||
.build(); |
||||
listenerEventDao.insert(listenerEvent); |
||||
listenerEventDao.updateListenerEvent(listenerEvent.getId(), AlertStatus.EXECUTION_SUCCESS, "test", new Date()); |
||||
assertThat(listenerEventDao.queryById(listenerEvent.getId()).getPostStatus()) |
||||
.isEqualTo(AlertStatus.EXECUTION_SUCCESS); |
||||
} |
||||
} |
@ -0,0 +1,105 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.registry.api.ha; |
||||
|
||||
import org.apache.dolphinscheduler.common.thread.ThreadUtils; |
||||
import org.apache.dolphinscheduler.registry.api.Event; |
||||
import org.apache.dolphinscheduler.registry.api.Registry; |
||||
|
||||
import java.util.List; |
||||
import java.util.concurrent.ScheduledExecutorService; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import com.google.common.collect.Lists; |
||||
|
||||
@Slf4j |
||||
public abstract class AbstractHAServer implements HAServer { |
||||
|
||||
private final Registry registry; |
||||
|
||||
private final String serverPath; |
||||
|
||||
private ServerStatus serverStatus; |
||||
|
||||
private final List<ServerStatusChangeListener> serverStatusChangeListeners; |
||||
|
||||
public AbstractHAServer(Registry registry, String serverPath) { |
||||
this.registry = registry; |
||||
this.serverPath = serverPath; |
||||
this.serverStatus = ServerStatus.STAND_BY; |
||||
this.serverStatusChangeListeners = Lists.newArrayList(new DefaultServerStatusChangeListener()); |
||||
} |
||||
|
||||
@Override |
||||
public void start() { |
||||
registry.subscribe(serverPath, event -> { |
||||
if (Event.Type.REMOVE.equals(event.type())) { |
||||
if (isActive() && !participateElection()) { |
||||
statusChange(ServerStatus.STAND_BY); |
||||
} |
||||
} |
||||
}); |
||||
ScheduledExecutorService electionSelectionThread = |
||||
ThreadUtils.newSingleDaemonScheduledExecutorService("election-selection-thread"); |
||||
electionSelectionThread.schedule(() -> { |
||||
if (isActive()) { |
||||
return; |
||||
} |
||||
if (participateElection()) { |
||||
statusChange(ServerStatus.ACTIVE); |
||||
} |
||||
}, 10, TimeUnit.SECONDS); |
||||
} |
||||
|
||||
@Override |
||||
public boolean isActive() { |
||||
return ServerStatus.ACTIVE.equals(getServerStatus()); |
||||
} |
||||
|
||||
@Override |
||||
public boolean participateElection() { |
||||
return registry.acquireLock(serverPath, 3_000); |
||||
} |
||||
|
||||
@Override |
||||
public void addServerStatusChangeListener(ServerStatusChangeListener listener) { |
||||
serverStatusChangeListeners.add(listener); |
||||
} |
||||
|
||||
@Override |
||||
public ServerStatus getServerStatus() { |
||||
return serverStatus; |
||||
} |
||||
|
||||
@Override |
||||
public void shutdown() { |
||||
if (isActive()) { |
||||
registry.releaseLock(serverPath); |
||||
} |
||||
} |
||||
|
||||
private void statusChange(ServerStatus targetStatus) { |
||||
synchronized (this) { |
||||
ServerStatus originStatus = serverStatus; |
||||
serverStatus = targetStatus; |
||||
serverStatusChangeListeners.forEach(listener -> listener.change(originStatus, serverStatus)); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,42 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.registry.api.ha; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public abstract class AbstractServerStatusChangeListener implements ServerStatusChangeListener { |
||||
|
||||
@Override |
||||
public void change(HAServer.ServerStatus originStatus, HAServer.ServerStatus currentStatus) { |
||||
log.info("The status change from {} to {}.", originStatus, currentStatus); |
||||
if (originStatus == HAServer.ServerStatus.ACTIVE) { |
||||
if (currentStatus == HAServer.ServerStatus.STAND_BY) { |
||||
changeToStandBy(); |
||||
} |
||||
} else if (originStatus == HAServer.ServerStatus.STAND_BY) { |
||||
if (currentStatus == HAServer.ServerStatus.ACTIVE) { |
||||
changeToActive(); |
||||
} |
||||
} |
||||
} |
||||
|
||||
public abstract void changeToActive(); |
||||
|
||||
public abstract void changeToStandBy(); |
||||
} |
@ -0,0 +1,34 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.registry.api.ha; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class DefaultServerStatusChangeListener extends AbstractServerStatusChangeListener { |
||||
|
||||
@Override |
||||
public void changeToActive() { |
||||
log.info("The status is active now."); |
||||
} |
||||
|
||||
@Override |
||||
public void changeToStandBy() { |
||||
log.info("The status is standby now."); |
||||
} |
||||
} |
@ -0,0 +1,68 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.registry.api.ha; |
||||
|
||||
/** |
||||
* Interface for HA server, used to select a active server from multiple servers. |
||||
* In HA mode, there are multiple servers, only one server is active, others are standby. |
||||
*/ |
||||
public interface HAServer { |
||||
|
||||
/** |
||||
* Start the server. |
||||
*/ |
||||
void start(); |
||||
|
||||
/** |
||||
* Judge whether the server is active. |
||||
* |
||||
* @return true if the current server is active. |
||||
*/ |
||||
boolean isActive(); |
||||
|
||||
/** |
||||
* Participate in the election of active server, this method will block until the server is active. |
||||
*/ |
||||
boolean participateElection(); |
||||
|
||||
/** |
||||
* Add a listener to listen to the status change of the server. |
||||
* |
||||
* @param listener listener to add. |
||||
*/ |
||||
void addServerStatusChangeListener(ServerStatusChangeListener listener); |
||||
|
||||
/** |
||||
* Get the status of the server. |
||||
* |
||||
* @return the status of the server. |
||||
*/ |
||||
ServerStatus getServerStatus(); |
||||
|
||||
/** |
||||
* Shutdown the server, release resources. |
||||
*/ |
||||
void shutdown(); |
||||
|
||||
enum ServerStatus { |
||||
ACTIVE, |
||||
STAND_BY, |
||||
; |
||||
} |
||||
|
||||
} |
@ -0,0 +1,24 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.registry.api.ha; |
||||
|
||||
public interface ServerStatusChangeListener { |
||||
|
||||
void change(HAServer.ServerStatus originStatus, HAServer.ServerStatus currentStatus); |
||||
|
||||
} |
Loading…
Reference in new issue