You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ji...@apache.org on 2019/02/11 18:42:09 UTC

[incubator-pinot] branch master updated: [TE] yaml - onboarding tasks (#3790)

This is an automated email from the ASF dual-hosted git repository.

jihao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 8b90a54  [TE] yaml - onboarding tasks (#3790)
8b90a54 is described below

commit 8b90a545115a1251182de2e6c081ba6415a79633
Author: Jihao Zhang <ji...@linkedin.com>
AuthorDate: Mon Feb 11 10:42:04 2019 -0800

    [TE] yaml - onboarding tasks (#3790)
    
    Run yaml onboarding tasks after the alert is set up. The onboarding task replays with last month's data and retunes the pipeline against the replay result.
---
 .../pinot/thirdeye/anomaly/task/TaskConstants.java |   1 +
 .../thirdeye/anomaly/task/TaskInfoFactory.java     |   4 +
 .../thirdeye/anomaly/task/TaskRunnerFactory.java   |   4 +
 .../onboard/YamlOnboardingTaskInfo.java}           |  44 +++----
 .../onboard/YamlOnboardingTaskRunner.java          | 138 +++++++++++++++++++++
 .../validators/DetectionConfigValidator.java       |   2 +-
 .../yaml/YamlDetectionConfigTranslator.java        |   2 +-
 .../thirdeye/detection/yaml/YamlResource.java      |  50 +++++++-
 8 files changed, 215 insertions(+), 30 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java
index 05c54f8..746a43d 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java
@@ -24,6 +24,7 @@ public class TaskConstants {
   public enum TaskType {
     DETECTION,
     DETECTION_ALERT,
+    YAML_DETECTION_ONBOARD,
     ANOMALY_DETECTION,
     MERGE,
     // TODO: deprecate ALERT task type
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskInfoFactory.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskInfoFactory.java
index ecbe5db..ede19d7 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskInfoFactory.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskInfoFactory.java
@@ -30,6 +30,7 @@ import org.apache.pinot.thirdeye.completeness.checker.DataCompletenessTaskInfo;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineTaskInfo;
 import org.apache.pinot.thirdeye.detection.alert.DetectionAlertTaskInfo;
 import java.io.IOException;
+import org.apache.pinot.thirdeye.detection.onboard.YamlOnboardingTaskInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,6 +53,9 @@ public class TaskInfoFactory {
         case DETECTION_ALERT:
           taskInfo = OBJECT_MAPPER.readValue(taskInfoString, DetectionAlertTaskInfo.class);
           break;
+        case YAML_DETECTION_ONBOARD:
+          taskInfo = OBJECT_MAPPER.readValue(taskInfoString, YamlOnboardingTaskInfo.class);
+          break;
         case ANOMALY_DETECTION:
           taskInfo = OBJECT_MAPPER.readValue(taskInfoString, DetectionTaskInfo.class);
           break;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskRunnerFactory.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskRunnerFactory.java
index 410cd8e..deac910 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskRunnerFactory.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskRunnerFactory.java
@@ -28,6 +28,7 @@ import org.apache.pinot.thirdeye.anomaly.task.TaskConstants.TaskType;
 import org.apache.pinot.thirdeye.completeness.checker.DataCompletenessTaskRunner;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineTaskRunner;
 import org.apache.pinot.thirdeye.detection.alert.DetectionAlertTaskRunner;
+import org.apache.pinot.thirdeye.detection.onboard.YamlOnboardingTaskRunner;
 
 
 /**
@@ -44,6 +45,9 @@ public class TaskRunnerFactory {
       case DETECTION_ALERT:
         taskRunner = new DetectionAlertTaskRunner();
         break;
+      case YAML_DETECTION_ONBOARD:
+        taskRunner = new YamlOnboardingTaskRunner();
+        break;
       case ANOMALY_DETECTION:
         taskRunner = new DetectionTaskRunner();
         break;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskInfo.java
similarity index 57%
copy from thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java
copy to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskInfo.java
index 05c54f8..dc75194 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/task/TaskConstants.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskInfo.java
@@ -15,31 +15,31 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
 
-package org.apache.pinot.thirdeye.anomaly.task;
-
-public class TaskConstants {
-
-  public enum TaskType {
-    DETECTION,
-    DETECTION_ALERT,
-    ANOMALY_DETECTION,
-    MERGE,
-    // TODO: deprecate ALERT task type
-    ALERT,
-    ALERT2,
-    MONITOR,
-    DATA_COMPLETENESS,
-    CLASSIFICATION,
-    REPLAY
+package org.apache.pinot.thirdeye.detection.onboard;
+
+import org.apache.pinot.thirdeye.detection.DetectionPipelineTaskInfo;
+
+
+public class YamlOnboardingTaskInfo extends DetectionPipelineTaskInfo {
+  private long tuningWindowStart;
+  private long tuningWindowEnd;
+
+  public long getTuningWindowStart() {
+    return tuningWindowStart;
+  }
+
+  public void setTuningWindowStart(long tuningWindowStart) {
+    this.tuningWindowStart = tuningWindowStart;
+  }
+
+  public long getTuningWindowEnd() {
+    return tuningWindowEnd;
   }
 
-  public enum TaskStatus {
-    WAITING,
-    RUNNING,
-    COMPLETED,
-    FAILED,
-    TIMEOUT
+  public void setTuningWindowEnd(long tuningWindowEnd) {
+    this.tuningWindowEnd = tuningWindowEnd;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
new file mode 100644
index 0000000..5730b41
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/onboard/YamlOnboardingTaskRunner.java
@@ -0,0 +1,138 @@
+/*
+ * 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.pinot.thirdeye.detection.onboard;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.thirdeye.anomaly.task.TaskContext;
+import org.apache.pinot.thirdeye.anomaly.task.TaskInfo;
+import org.apache.pinot.thirdeye.anomaly.task.TaskResult;
+import org.apache.pinot.thirdeye.anomaly.task.TaskRunner;
+import org.apache.pinot.thirdeye.constant.AnomalyResultSource;
+import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
+import org.apache.pinot.thirdeye.datasource.loader.AggregationLoader;
+import org.apache.pinot.thirdeye.datasource.loader.DefaultAggregationLoader;
+import org.apache.pinot.thirdeye.datasource.loader.DefaultTimeSeriesLoader;
+import org.apache.pinot.thirdeye.datasource.loader.TimeSeriesLoader;
+import org.apache.pinot.thirdeye.detection.DataProvider;
+import org.apache.pinot.thirdeye.detection.DefaultDataProvider;
+import org.apache.pinot.thirdeye.detection.DetectionPipeline;
+import org.apache.pinot.thirdeye.detection.DetectionPipelineLoader;
+import org.apache.pinot.thirdeye.detection.DetectionPipelineResult;
+import org.apache.pinot.thirdeye.detection.yaml.YamlDetectionConfigTranslator;
+import org.apache.pinot.thirdeye.detection.yaml.YamlDetectionTranslatorLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
+
+
+/**
+ * The task runner to run yaml onboarding task after a new detection is set up
+ * It will replay the detection pipeline and the re-tune the pipeline.
+ * Because for some pipeline component, tuning is depend on replay result
+ */
+public class YamlOnboardingTaskRunner implements TaskRunner {
+  private static final Logger LOG = LoggerFactory.getLogger(YamlOnboardingTaskRunner.class);
+  private final DetectionConfigManager detectionDAO;
+  private final MergedAnomalyResultManager anomalyDAO;
+  private final DetectionPipelineLoader loader;
+  private final DataProvider provider;
+  private final YamlDetectionTranslatorLoader translatorLoader;
+  private final Yaml yaml;
+
+
+  public YamlOnboardingTaskRunner() {
+    this.loader = new DetectionPipelineLoader();
+    this.detectionDAO = DAORegistry.getInstance().getDetectionConfigManager();
+    this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.translatorLoader = new YamlDetectionTranslatorLoader();
+    this.yaml = new Yaml();
+
+    MetricConfigManager metricDAO = DAORegistry.getInstance().getMetricConfigDAO();
+    DatasetConfigManager datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
+    EventManager eventDAO = DAORegistry.getInstance().getEventDAO();
+
+    TimeSeriesLoader timeseriesLoader =
+        new DefaultTimeSeriesLoader(metricDAO, datasetDAO,
+            ThirdEyeCacheRegistry.getInstance().getQueryCache());
+
+    AggregationLoader aggregationLoader =
+        new DefaultAggregationLoader(metricDAO, datasetDAO,
+            ThirdEyeCacheRegistry.getInstance().getQueryCache(),
+            ThirdEyeCacheRegistry.getInstance().getDatasetMaxDataTimeCache());
+
+    this.provider = new DefaultDataProvider(metricDAO, datasetDAO, eventDAO, this.anomalyDAO,
+        timeseriesLoader, aggregationLoader, this.loader);
+  }
+
+  @Override
+  public List<TaskResult> execute(TaskInfo taskInfo, TaskContext taskContext) throws Exception {
+    YamlOnboardingTaskInfo info = (YamlOnboardingTaskInfo) taskInfo;
+    LOG.info("Running yaml detection onboarding task for id {}", info.getConfigId());
+
+    // replay the detection pipeline
+    DetectionConfigDTO config = this.detectionDAO.findById(info.getConfigId());
+    if (config == null) {
+      throw new IllegalArgumentException(String.format("Could not resolve config id %d", info.getConfigId()));
+    }
+
+    DetectionPipeline pipeline = this.loader.from(this.provider, config, info.getStart(), info.getEnd());
+    DetectionPipelineResult result = pipeline.run();
+
+    if (result.getLastTimestamp() < 0) {
+      return Collections.emptyList();
+    }
+
+    config.setLastTimestamp(result.getLastTimestamp());
+    this.detectionDAO.update(config);
+
+    for (MergedAnomalyResultDTO anomaly : result.getAnomalies()) {
+      anomaly.setAnomalyResultSource(AnomalyResultSource.ANOMALY_REPLAY);
+      this.anomalyDAO.save(anomaly);
+      if (anomaly.getId() == null) {
+        LOG.warn("Could not store anomaly:\n{}", anomaly);
+      }
+    }
+
+    // re-tune the detection pipeline because tuning is depend on replay result. e.g. algorithm-based alert filter
+    YamlDetectionConfigTranslator translator =
+        this.translatorLoader.from((Map<String, Object>) this.yaml.load(config.getYaml()), this.provider);
+
+    DetectionConfigDTO newDetectionConfig =
+        translator.withTuningWindow(info.getTuningWindowStart(), info.getTuningWindowEnd())
+        .withExistingDetectionConfig(config)
+        .generateDetectionConfig();
+
+    this.detectionDAO.save(newDetectionConfig);
+
+    LOG.info("Yaml detection onboarding task for id {} completed", info.getConfigId());
+    return Collections.emptyList();
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
index bcd7586..b94f70d 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/validators/DetectionConfigValidator.java
@@ -85,7 +85,7 @@ public class DetectionConfigValidator extends ConfigValidator {
       // set id back
       detectionConfig.setId(id);
     } catch (Exception e) {
-      throw new ValidationException("Semantic error while initializing the detection pipeline.");
+      throw new ValidationException("Semantic error while initializing the detection pipeline.", e.getMessage());
     }
   }
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java
index 52ccd4b..5d508e9 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionConfigTranslator.java
@@ -54,7 +54,7 @@ public abstract class YamlDetectionConfigTranslator {
     this.existingComponentSpecs = new HashMap<>();
   }
 
-  public YamlDetectionConfigTranslator withTrainingWindow(long startTime, long endTime) {
+  public YamlDetectionConfigTranslator withTuningWindow(long startTime, long endTime) {
     this.startTime = startTime;
     this.endTime = endTime;
     return this;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
index 41c84d6..0a8aba0 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlResource.java
@@ -45,6 +45,7 @@ import javax.ws.rs.core.Response;
 import javax.xml.bind.ValidationException;
 import org.apache.commons.collections.MapUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.anomaly.task.TaskConstants;
 import org.apache.pinot.thirdeye.api.Constants;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
@@ -52,8 +53,10 @@ import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.EventManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.TaskManager;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.TaskDTO;
 import org.apache.pinot.thirdeye.datalayer.util.Predicate;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
@@ -67,6 +70,7 @@ import org.apache.pinot.thirdeye.detection.DefaultDataProvider;
 import org.apache.pinot.thirdeye.detection.DetectionPipeline;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineLoader;
 import org.apache.pinot.thirdeye.detection.DetectionPipelineResult;
+import org.apache.pinot.thirdeye.detection.onboard.YamlOnboardingTaskInfo;
 import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
 import org.apache.pinot.thirdeye.detection.validators.SubscriptionConfigValidator;
 import org.slf4j.Logger;
@@ -81,6 +85,7 @@ public class YamlResource {
   private static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   private static final String PROP_SUBS_GROUP_NAME = "subscriptionGroupName";
+  private static final long ONBOARDING_REPLAY_LOOKBACK = TimeUnit.DAYS.toMillis(30);
 
   private final DetectionConfigManager detectionConfigDAO;
   private final DetectionAlertConfigManager detectionAlertConfigDAO;
@@ -93,6 +98,7 @@ public class YamlResource {
   private final DatasetConfigManager datasetDAO;
   private final EventManager eventDAO;
   private final MergedAnomalyResultManager anomalyDAO;
+  private final TaskManager taskDAO;
   private final DetectionPipelineLoader loader;
   private final Yaml yaml;
 
@@ -107,6 +113,7 @@ public class YamlResource {
     this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
     this.eventDAO = DAORegistry.getInstance().getEventDAO();
     this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.taskDAO = DAORegistry.getInstance().getTaskDAO();
     this.yaml = new Yaml();
 
     TimeSeriesLoader timeseriesLoader =
@@ -128,22 +135,52 @@ public class YamlResource {
   /*
    * Build the detection config from a yaml.
    */
-  private DetectionConfigDTO buildDetectionConfigFromYaml(long startTime, long endTime, Map<String, Object> yamlConfig,
+  private DetectionConfigDTO buildDetectionConfigFromYaml(long tuningStartTime, long tuningEndTime, Map<String, Object> yamlConfig,
       DetectionConfigDTO existingDetectionConfig) throws Exception {
 
     // Configure the tuning window
-    if (startTime == 0L && endTime == 0L) {
+    if (tuningStartTime == 0L && tuningEndTime == 0L) {
       // default tuning window 28 days
-      endTime = System.currentTimeMillis();
-      startTime = endTime - TimeUnit.DAYS.toMillis(28);
+      tuningEndTime = System.currentTimeMillis();
+      tuningStartTime = tuningEndTime - TimeUnit.DAYS.toMillis(28);
     }
 
     YamlDetectionConfigTranslator translator = this.translatorLoader.from(yamlConfig, this.provider);
-    return translator.withTrainingWindow(startTime, endTime)
+    return translator.withTuningWindow(tuningStartTime, tuningEndTime)
         .withExistingDetectionConfig(existingDetectionConfig)
         .generateDetectionConfig();
   }
 
+  /*
+   * Create a yaml onboarding task. It runs 1 month replay and re-tune the pipeline.
+   */
+  private void createYamlOnboardingTask(long configId, long tuningWindowStart, long tuningWindowEnd) throws Exception {
+    YamlOnboardingTaskInfo info = new YamlOnboardingTaskInfo();
+    info.setConfigId(configId);
+    if (tuningWindowStart == 0L && tuningWindowEnd == 0L) {
+      // default tuning window 28 days
+      tuningWindowEnd = System.currentTimeMillis();
+      tuningWindowStart = tuningWindowEnd - TimeUnit.DAYS.toMillis(28);
+    }
+    info.setTuningWindowStart(tuningWindowStart);
+    info.setTuningWindowEnd(tuningWindowEnd);
+    info.setEnd(System.currentTimeMillis());
+    info.setStart(info.getEnd() - ONBOARDING_REPLAY_LOOKBACK);
+
+    String taskInfoJson = OBJECT_MAPPER.writeValueAsString(info);
+    String jobName = String.format("%s_%d", TaskConstants.TaskType.YAML_DETECTION_ONBOARD, configId);
+
+    TaskDTO taskDTO = new TaskDTO();
+    taskDTO.setTaskType(TaskConstants.TaskType.YAML_DETECTION_ONBOARD);
+    taskDTO.setJobName(jobName);
+    taskDTO.setStatus(TaskConstants.TaskStatus.WAITING);
+    taskDTO.setTaskInfo(taskInfoJson);
+
+    long taskId = this.taskDAO.save(taskDTO);
+    LOG.info("Created yaml detection onboarding task {} with taskId {}", taskDTO, taskId);
+  }
+
+
   @POST
   @Path("/create-alert")
   @Produces(MediaType.APPLICATION_JSON)
@@ -195,7 +232,8 @@ public class YamlResource {
       return Response.serverError().entity(response.getEntity()).build();
     }
     long alertId = Long.parseLong(ConfigUtils.getMap(response.getEntity()).get("detectionAlertConfigId").toString());
-
+    // create an yaml onboarding task to run replay and tuning
+    createYamlOnboardingTask(detectionConfigId, startTime, endTime);
     return Response.ok().entity(ImmutableMap.of("detectionConfigId", detectionConfigId, "detectionAlertConfigId", alertId)).build();
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org