You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ak...@apache.org on 2019/06/19 23:42:25 UTC

[incubator-pinot] branch master updated: [TE] Entity Yaml (Composite Alert) Translator (#4333)

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

akshayrai09 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 5b34223  [TE] Entity Yaml (Composite Alert) Translator (#4333)
5b34223 is described below

commit 5b34223067aea8f8da9bbaa3373bd07e247a66ec
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Wed Jun 19 16:42:20 2019 -0700

    [TE] Entity Yaml (Composite Alert) Translator (#4333)
    
    Changes:
    - Introduced 2 types of alerts - METRIC_ALERT and COMPOSITE_ALERT (refer yaml examples)
    - Modified and refactored the DetectionConfigTranslator to work for composite alerts by recursively calling the metric alert translator
---
 .../thirdeye/detection/components/MockGrouper.java |   0
 .../thirdeye/detection/spec/MockGrouperSpec.java   |   0
 .../thirdeye/detection/wrapper/GrouperWrapper.java |  18 ++-
 .../yaml/translator/DetectionConfigTranslator.java | 136 +++++++++++++++------
 .../translator/DetectionMetricAttributeHolder.java |  97 +++++++++++++++
 .../yaml/translator/DetectionMetricProperties.java |  62 ++++++++++
 .../translator/DetectionConfigTranslatorTest.java  |  18 +++
 .../validators/entity-pipeline-config-2.yaml       |   2 +-
 .../compositePipelineTranslatorTestResult-4.json   |  35 ++++++
 .../compositePipelineTranslatorTestResult-5.json   |  91 ++++++++++++++
 .../yaml/translator/pipeline-config-4.yaml         |  26 ++++
 .../translator/pipeline-config-5.yaml}             |  13 +-
 12 files changed, 444 insertions(+), 54 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockGrouper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MockGrouper.java
similarity index 100%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/components/MockGrouper.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/components/MockGrouper.java
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockGrouperSpec.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spec/MockGrouperSpec.java
similarity index 100%
rename from thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/spec/MockGrouperSpec.java
rename to thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/spec/MockGrouperSpec.java
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
index c9b31e8..a839b7f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/wrapper/GrouperWrapper.java
@@ -21,12 +21,14 @@ package org.apache.pinot.thirdeye.detection.wrapper;
 
 import com.google.common.base.Preconditions;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.collections4.MapUtils;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.EvaluationDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
 import org.apache.pinot.thirdeye.detection.ConfigUtils;
 import org.apache.pinot.thirdeye.detection.DataProvider;
@@ -71,29 +73,37 @@ public class GrouperWrapper extends DetectionPipeline {
   @Override
   public final DetectionPipelineResult run() throws Exception {
     List<MergedAnomalyResultDTO> candidates = new ArrayList<>();
+    Map<String, Object> diagnostics = new HashMap<>();
+    List<MergedAnomalyResultDTO> generated = new ArrayList<>();
+    List<PredictionResult> predictionResults = new ArrayList<>();
+    List<EvaluationDTO> evaluations = new ArrayList<>();
 
     Set<Long> lastTimeStamps = new HashSet<>();
-    List<PredictionResult> predictionResults = new ArrayList<>();
     for (Map<String, Object> properties : this.nestedProperties) {
       DetectionConfigDTO nestedConfig = new DetectionConfigDTO();
 
       Preconditions.checkArgument(properties.containsKey(PROP_CLASS_NAME), "Nested missing " + PROP_CLASS_NAME);
+
       nestedConfig.setId(this.config.getId());
       nestedConfig.setName(this.config.getName());
       nestedConfig.setDescription(this.config.getDescription());
       nestedConfig.setProperties(properties);
-
+      nestedConfig.setComponents(this.config.getComponents());
       DetectionPipeline pipeline = this.provider.loadPipeline(nestedConfig, this.startTime, this.endTime);
 
       DetectionPipelineResult intermediate = pipeline.run();
       lastTimeStamps.add(intermediate.getLastTimestamp());
-      predictionResults.addAll(intermediate.getPredictions());
 
+      generated.addAll(intermediate.getAnomalies());
+      predictionResults.addAll(intermediate.getPredictions());
+      evaluations.addAll(intermediate.getEvaluations());
+      diagnostics.putAll(intermediate.getDiagnostics());
       candidates.addAll(intermediate.getAnomalies());
     }
 
     List<MergedAnomalyResultDTO> anomalies = this.grouper.group(candidates);
 
-    return new DetectionPipelineResult(anomalies, DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps), predictionResults);
+    return new DetectionPipelineResult(anomalies, DetectionUtils.consolidateNestedLastTimeStamps(lastTimeStamps),
+        predictionResults, evaluations).setDiagnostics(diagnostics);
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslator.java
index 3ecd1d0..7e76d5b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslator.java
@@ -19,6 +19,7 @@
 
 package org.apache.pinot.thirdeye.detection.yaml.translator;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -38,6 +39,7 @@ import org.apache.pinot.thirdeye.detection.DataProvider;
 import org.apache.pinot.thirdeye.detection.DetectionUtils;
 import org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper;
 import org.apache.pinot.thirdeye.detection.annotation.registry.DetectionRegistry;
+import org.apache.pinot.thirdeye.detection.components.MockGrouper;
 import org.apache.pinot.thirdeye.detection.validators.DetectionConfigValidator;
 import org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper;
 import org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper;
@@ -127,8 +129,6 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
   private static final String PROP_CRON = "cron";
   private static final String PROP_FILTER = "filter";
   private static final String PROP_FILTERS = "filters";
-  private static final String PROP_METRIC = "metric";
-  private static final String PROP_DATASET = "dataset";
   private static final String PROP_TYPE = "type";
   private static final String PROP_CLASS_NAME = "className";
   private static final String PROP_PARAMS = "params";
@@ -157,6 +157,11 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
   private static final String PROP_DESC_NAME = "description";
   private static final String PROP_ACTIVE = "active";
 
+  private static final String PROP_ALERTS = "alerts";
+  private static final String COMPOSITE_ALERT = "COMPOSITE_ALERT";
+  private static final String METRIC_ALERT = "METRIC_ALERT";
+
+
   private static final DetectionRegistry DETECTION_REGISTRY = DetectionRegistry.getInstance();
   static {
     // do not tune for alerts migrated from legacy anomaly function.
@@ -169,6 +174,7 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
 
   private final Map<String, Object> components = new HashMap<>();
   private DataProvider dataProvider;
+  private DetectionMetricAttributeHolder metricAttributesMap;
 
   public DetectionConfigTranslator(String yamlConfig, DataProvider provider) {
     this(yamlConfig, provider, new DetectionConfigValidator(provider));
@@ -177,28 +183,23 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
   public DetectionConfigTranslator(String yamlConfig, DataProvider provider, DetectionConfigValidator validator) {
     super(yamlConfig, validator);
     this.dataProvider = provider;
+    this.metricAttributesMap = new DetectionMetricAttributeHolder(provider);
   }
 
-  @Override
-  DetectionConfigDTO translateConfig(Map<String, Object> yamlConfigMap) throws IllegalArgumentException {
-    Map<String, Collection<String>> dimensionFiltersMap = ConfigUtils.getMap(yamlConfigMap.get(PROP_FILTERS));
-
-    MetricConfigDTO metricConfig = this.dataProvider.fetchMetric(MapUtils.getString(yamlConfigMap, PROP_METRIC),
-        MapUtils.getString(yamlConfigMap, PROP_DATASET));
-    DatasetConfigDTO datasetConfig = this.dataProvider.fetchDatasets(Collections.singletonList(metricConfig.getDataset()))
-        .get(metricConfig.getDataset());
-    String metricUrn = MetricEntity.fromMetric(dimensionFiltersMap, metricConfig.getId()).getUrn();
+  private Map<String, Object> translateMetricAlert(Map<String, Object> metricAlertConfigMap) {
+    DatasetConfigDTO datasetConfigDTO = metricAttributesMap.fetchDataset(metricAlertConfigMap);
+    Map<String, Collection<String>> dimensionFiltersMap = ConfigUtils.getMap(metricAlertConfigMap.get(PROP_FILTERS));
+    String metricUrn = MetricEntity.fromMetric(dimensionFiltersMap, metricAttributesMap.fetchMetric(metricAlertConfigMap).getId()).getUrn();
+    Map<String, Object> mergerProperties = ConfigUtils.getMap(metricAlertConfigMap.get(PROP_MERGER));
 
-    String detectionCronInYaml = MapUtils.getString(yamlConfigMap, PROP_CRON);
-    String cron = (detectionCronInYaml == null) ? buildCron(datasetConfig.bucketTimeGranularity()) : detectionCronInYaml;
-    Map<String, Object> mergerProperties = ConfigUtils.getMap(yamlConfigMap.get(PROP_MERGER));
-
-    List<Map<String, Object>> ruleYamls = getList(yamlConfigMap.get(PROP_RULES));
+    // Translate all the rules
+    List<Map<String, Object>> ruleYamls = getList(metricAlertConfigMap.get(PROP_RULES));
     List<Map<String, Object>> nestedPipelines = new ArrayList<>();
     for (Map<String, Object> ruleYaml : ruleYamls) {
       List<Map<String, Object>> filterYamls = ConfigUtils.getList(ruleYaml.get(PROP_FILTER));
       List<Map<String, Object>> detectionYamls = ConfigUtils.getList(ruleYaml.get(PROP_DETECTION));
-      List<Map<String, Object>> detectionProperties = buildListOfMergeWrapperProperties(detectionYamls, mergerProperties, datasetConfig.bucketTimeGranularity());
+      List<Map<String, Object>> detectionProperties = buildListOfMergeWrapperProperties(detectionYamls, mergerProperties,
+          datasetConfigDTO.bucketTimeGranularity());
       if (filterYamls.isEmpty()) {
         nestedPipelines.addAll(detectionProperties);
       } else {
@@ -210,16 +211,89 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
         nestedPipelines.addAll(filterNestedProperties);
       }
     }
+
+    // Wrap with dimension exploration properties
     Map<String, Object> dimensionWrapperProperties = buildDimensionWrapperProperties(
-        yamlConfigMap, dimensionFiltersMap, metricUrn, datasetConfig.getDataset());
+        metricAlertConfigMap, dimensionFiltersMap, metricUrn, datasetConfigDTO.getDataset());
     Map<String, Object> properties = buildWrapperProperties(
         ChildKeepingMergeWrapper.class.getName(),
         Collections.singletonList(buildWrapperProperties(DimensionWrapper.class.getName(), nestedPipelines, dimensionWrapperProperties)),
         mergerProperties);
 
-    List<Map<String, Object>> grouperYamls = getList(yamlConfigMap.get(PROP_GROUPER));
+    // Wrap with metric level grouper, restricting to only 1 grouper
+    List<Map<String, Object>> grouperYamls = getList(metricAlertConfigMap.get(PROP_GROUPER));
     if (!grouperYamls.isEmpty()) {
-      properties = buildGroupWrapperProperties(grouperYamls.get(0), properties);
+      properties = buildGroupWrapperProperties(grouperYamls.get(0), Collections.singletonList(properties));
+    }
+
+    return properties;
+  }
+
+  private Map<String, Object> translateCompositeAlert(Map<String, Object> compositeAlertConfigMap) {
+    Map<String, Object> properties;
+
+    // Recursively translate all the sub-alerts
+    List<Map<String, Object>> subDetectionYamls = ConfigUtils.getList(compositeAlertConfigMap.get(PROP_ALERTS));
+    List<Map<String, Object>> nestedPropertiesList = new ArrayList<>();
+    for (Map<String, Object> subDetectionYaml : subDetectionYamls) {
+      Map<String, Object> subProps;
+      if (subDetectionYaml.containsKey(PROP_TYPE) && subDetectionYaml.get(PROP_TYPE).equals(COMPOSITE_ALERT)) {
+        subProps = translateCompositeAlert(subDetectionYaml);
+      } else {
+        subProps = translateMetricAlert(subDetectionYaml);
+      }
+
+      nestedPropertiesList.add(subProps);
+    }
+
+    // Wrap the entity level grouper, only 1 grouper is supported now
+    List<Map<String, Object>> grouperProps = ConfigUtils.getList(compositeAlertConfigMap.get(PROP_GROUPER));
+    if (!grouperProps.isEmpty()) {
+      properties = buildGroupWrapperProperties(grouperProps.get(0), nestedPropertiesList);
+    } else {
+      Map<String, Object> defaultGrouper = new HashMap<>();
+      defaultGrouper.put(PROP_TYPE, "MOCK_GROUPER");
+      defaultGrouper.put(PROP_NAME, "Default grouper");
+      properties = buildGroupWrapperProperties(defaultGrouper, nestedPropertiesList);
+    }
+
+    // Wrap the entity level merger
+    Map<String, Object> mergerProperties = ConfigUtils.getMap(compositeAlertConfigMap.get(PROP_MERGER));
+    properties = buildWrapperProperties(
+        ChildKeepingMergeWrapper.class.getName(),
+        Collections.singletonList(properties),
+        mergerProperties);
+
+    return properties;
+  }
+
+  @Override
+  DetectionConfigDTO translateConfig(Map<String, Object> yamlConfigMap) throws IllegalArgumentException {
+    // Hack to support 'detectionName' attribute at root level and 'name' attribute elsewhere
+    // We consistently use 'name' as a convention to define the sub-alerts. However, at the root
+    // level, as a convention, we will use 'detectionName' which defines the name of the complete alert.
+    String alertName = MapUtils.getString(yamlConfigMap, PROP_DETECTION_NAME);
+    yamlConfigMap.put(PROP_NAME, alertName);
+
+    // By default if 'type' is not specified, we assume it as a METRIC_ALERT
+    if (!yamlConfigMap.containsKey(PROP_TYPE)) {
+      yamlConfigMap.put(PROP_TYPE, METRIC_ALERT);
+    }
+
+    // Translate config depending on the type (METRIC_ALERT OR COMPOSITE_ALERT)
+    Map<String, Object> properties;
+    String cron;
+    if (yamlConfigMap.get(PROP_TYPE).equals(COMPOSITE_ALERT)) {
+      properties = translateCompositeAlert(yamlConfigMap);
+
+      // TODO: discuss strategy for default cron
+      Preconditions.checkArgument(yamlConfigMap.containsKey(PROP_CRON), "Missing property (" + PROP_CRON + ") in alert");
+      cron = MapUtils.getString(yamlConfigMap, PROP_CRON);
+    } else {
+      // The legacy type 'COMPOSITE' will be treated as a metric alert along with the new convention METRIC_ALERT.
+      // This is applicable only at the root level to maintain backward compatibility.
+      properties = translateMetricAlert(yamlConfigMap);
+      cron = metricAttributesMap.fetchCron(yamlConfigMap);
     }
 
     return generateDetectionConfig(yamlConfigMap, properties, this.components, cron);
@@ -281,10 +355,10 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
     return properties;
   }
 
-  private Map<String, Object> buildGroupWrapperProperties(Map<String, Object> grouperYaml, Map<String, Object> nestedProps) {
+  private Map<String, Object> buildGroupWrapperProperties(Map<String, Object> grouperYaml, List<Map<String, Object>> nestedProps) {
     Map<String, Object> properties = new HashMap<>();
     properties.put(PROP_CLASS_NAME, GrouperWrapper.class.getName());
-    properties.put(PROP_NESTED, Collections.singletonList(nestedProps));
+    properties.put(PROP_NESTED, nestedProps);
 
     String grouperType = MapUtils.getString(grouperYaml, PROP_TYPE);
     String grouperName = MapUtils.getString(grouperYaml, PROP_NAME);
@@ -389,24 +463,6 @@ public class DetectionConfigTranslator extends ConfigTranslator<DetectionConfigD
     return properties;
   }
 
-  //  Default schedule:
-  //  minute granularity: every 15 minutes, starts at 0 minute
-  //  hourly: every hour, starts at 0 minute
-  //  daily: every day, starts at 2 pm UTC
-  //  others: every day, start at 12 am UTC
-  private String buildCron(TimeGranularity timegranularity) {
-    switch (timegranularity.getUnit()) {
-      case MINUTES:
-        return "0 0/15 * * * ? *";
-      case HOURS:
-        return "0 0 * * * ? *";
-      case DAYS:
-        return "0 0 14 * * ? *";
-      default:
-        return "0 0 0 * * ?";
-    }
-  }
-
   private void buildComponentSpec(Map<String, Object> yamlConfig, String type, String componentRefKey) {
     Map<String, Object> componentSpecs = new HashMap<>();
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricAttributeHolder.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricAttributeHolder.java
new file mode 100644
index 0000000..b293937
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricAttributeHolder.java
@@ -0,0 +1,97 @@
+/*
+ * 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.yaml.translator;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.collections4.MapUtils;
+import org.apache.pinot.thirdeye.common.time.TimeGranularity;
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+import org.apache.pinot.thirdeye.detection.DataProvider;
+import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
+
+
+/**
+ * A data holder to store the processed information per metric
+ */
+class DetectionMetricAttributeHolder {
+
+  private static final String PROP_METRIC = "metric";
+  private static final String PROP_DATASET = "dataset";
+  private static final String PROP_CRON = "cron";
+
+  private final Map<String, DetectionMetricProperties> metricAttributesMap = new HashMap<>();
+  private final DataProvider dataProvider;
+
+  DetectionMetricAttributeHolder(DataProvider provider) {
+    this.dataProvider = provider;
+  }
+
+  private String loadMetricCache(Map<String, Object> metricAlertConfigMap) {
+    String metricName = MapUtils.getString(metricAlertConfigMap, PROP_METRIC);
+    String datasetName = MapUtils.getString(metricAlertConfigMap, PROP_DATASET);
+    String cron = MapUtils.getString(metricAlertConfigMap, PROP_CRON);
+    String metricAliasKey = ThirdEyeUtils.constructMetricAlias(datasetName, metricName);
+    if (metricAttributesMap.containsKey(metricAliasKey)) {
+      return metricAliasKey;
+    }
+
+    MetricConfigDTO metricConfig = this.dataProvider.fetchMetric(metricName, datasetName);
+    DatasetConfigDTO datasetConfig = this.dataProvider.fetchDatasets(Collections.singletonList(metricConfig.getDataset()))
+        .get(metricConfig.getDataset());
+    cron = cron == null ? buildCron(datasetConfig.bucketTimeGranularity()) : cron;
+
+    metricAttributesMap.put(metricAliasKey, new DetectionMetricProperties(cron, metricConfig, datasetConfig));
+
+    return metricAliasKey;
+  }
+
+  DatasetConfigDTO fetchDataset(Map<String, Object> metricAlertConfigMap) {
+    return metricAttributesMap.get(loadMetricCache(metricAlertConfigMap)).getDatasetConfigDTO();
+  }
+
+  MetricConfigDTO fetchMetric(Map<String, Object> metricAlertConfigMap) {
+    return metricAttributesMap.get(loadMetricCache(metricAlertConfigMap)).getMetricConfigDTO();
+  }
+
+  String fetchCron(Map<String, Object> metricAlertConfigMap) {
+    return metricAttributesMap.get(loadMetricCache(metricAlertConfigMap)).getCron();
+  }
+
+  //  Default schedule:
+  //  minute granularity: every 15 minutes, starts at 0 minute
+  //  hourly: every hour, starts at 0 minute
+  //  daily: every day, starts at 2 pm UTC
+  //  others: every day, start at 12 am UTC
+  private String buildCron(TimeGranularity timegranularity) {
+    switch (timegranularity.getUnit()) {
+      case MINUTES:
+        return "0 0/15 * * * ? *";
+      case HOURS:
+        return "0 0 * * * ? *";
+      case DAYS:
+        return "0 0 14 * * ? *";
+      default:
+        return "0 0 0 * * ?";
+    }
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricProperties.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricProperties.java
new file mode 100644
index 0000000..41b2a7e
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionMetricProperties.java
@@ -0,0 +1,62 @@
+/*
+ * 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.yaml.translator;
+
+import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.dto.MetricConfigDTO;
+
+
+public class DetectionMetricProperties {
+
+  String cron;
+  DatasetConfigDTO datasetConfigDTO;
+  MetricConfigDTO metricConfigDTO;
+
+  DetectionMetricProperties(String cron, MetricConfigDTO metricConfig, DatasetConfigDTO datasetConfig) {
+    this.cron = cron;
+    this.metricConfigDTO = metricConfig;
+    this.datasetConfigDTO = datasetConfig;
+  }
+
+  public String getCron() {
+    return cron;
+  }
+
+  public void setCron(String cron) {
+    this.cron = cron;
+  }
+
+  public DatasetConfigDTO getDatasetConfigDTO() {
+    return datasetConfigDTO;
+  }
+
+  public void setDatasetConfigDTO(DatasetConfigDTO datasetConfigDTO) {
+    this.datasetConfigDTO = datasetConfigDTO;
+  }
+
+  public MetricConfigDTO getMetricConfigDTO() {
+    return metricConfigDTO;
+  }
+
+  public void setMetricConfigDTO(MetricConfigDTO metricConfigDTO) {
+    this.metricConfigDTO = metricConfigDTO;
+  }
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslatorTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslatorTest.java
index 3cf3576..f0ea30b 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslatorTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/yaml/translator/DetectionConfigTranslatorTest.java
@@ -104,4 +104,22 @@ public class DetectionConfigTranslatorTest {
     DetectionConfigTranslator translator = new DetectionConfigTranslator(yamlConfig, this.provider);
     translator.translate();
   }
+
+  @Test
+  public void testBuildEntityTranslationWithOneMetric() throws Exception {
+    String yamlConfig = IOUtils.toString(this.getClass().getResourceAsStream("pipeline-config-4.yaml"), "UTF-8");
+    DetectionConfigTranslator translator = new DetectionConfigTranslator(yamlConfig, this.provider);
+    DetectionConfigDTO result = translator.translate();
+    YamlTranslationResult expected = OBJECT_MAPPER.readValue(this.getClass().getResourceAsStream("compositePipelineTranslatorTestResult-4.json"), YamlTranslationResult.class);
+    Assert.assertEquals(result.getProperties(), expected.getProperties());
+  }
+
+  @Test
+  public void testBuildEntityTranslationWithMultipleMetrics() throws Exception {
+    String yamlConfig = IOUtils.toString(this.getClass().getResourceAsStream("pipeline-config-5.yaml"), "UTF-8");
+    DetectionConfigTranslator translator = new DetectionConfigTranslator(yamlConfig, this.provider);
+    DetectionConfigDTO result = translator.translate();
+    YamlTranslationResult expected = OBJECT_MAPPER.readValue(this.getClass().getResourceAsStream("compositePipelineTranslatorTestResult-5.json"), YamlTranslationResult.class);
+    Assert.assertEquals(result.getProperties(), expected.getProperties());
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
index bba31ed..fb82fe3 100644
--- a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
@@ -1,7 +1,7 @@
 detectionName: testPipeline
 description: My test pipeline
 type: COMPOSITE_ALERT
-
+cron: "0 0 14 * * ? *"
 alerts:
   - type: METRIC_ALERT
     name: metric alert on test_metric
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-4.json b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-4.json
new file mode 100644
index 0000000..2cc7cce
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-4.json
@@ -0,0 +1,35 @@
+{
+  "properties": {
+    "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+    "nested": [{
+      "grouper": "$Default grouper:MOCK_GROUPER",
+      "className": "org.apache.pinot.thirdeye.detection.wrapper.GrouperWrapper",
+      "nested": [{
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+        "nested": [{
+          "nestedMetricUrns": ["thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3"],
+          "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+          "metricUrn": "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
+          "nested": [{
+            "baselineValueProvider": "$maxThreshold_1:THRESHOLD",
+            "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+            "nested": [{
+              "bucketPeriod": "P1D",
+              "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+            }],
+            "detector": "$maxThreshold_1:THRESHOLD"
+          }],
+          "minContribution": 0.05,
+          "dimensions": ["D1", "D2"]
+        }]
+      }]
+    }]
+  },
+  "components": {
+    "rule1:THRESHOLD": {
+      "max": 100,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
+    }
+  },
+  "cron": "0 0 14 * * ? *"
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-5.json b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-5.json
new file mode 100644
index 0000000..2a84f5a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/compositePipelineTranslatorTestResult-5.json
@@ -0,0 +1,91 @@
+{
+  "properties": {
+    "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+    "nested": [{
+      "grouper": "$Default grouper:MOCK_GROUPER",
+      "className": "org.apache.pinot.thirdeye.detection.wrapper.GrouperWrapper",
+      "nested": [{
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+        "nested": [{
+          "nestedMetricUrns": ["thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3"],
+          "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+          "metricUrn": "thirdeye:metric:1:D1%3Dv1:D1%3Dv2:D2%3Dv3",
+          "nested": [{
+            "filter": "$thresholdFilter_2:THRESHOLD_RULE_FILTER",
+            "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
+            "nested": [{
+              "filter": "$thresholdFilter_1:THRESHOLD_RULE_FILTER",
+              "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyFilterWrapper",
+              "nested": [{
+                "baselineValueProvider": "$maxThreshold_1:THRESHOLD",
+                "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+                "nested": [{
+                  "bucketPeriod": "P1D",
+                  "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+                }],
+                "detector": "$maxThreshold_1:THRESHOLD"
+              }]
+            }]
+          },
+            {
+              "baselineValueProvider": "$maxThreshold_2:THRESHOLD",
+              "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+              "nested": [{
+                "bucketPeriod": "P1D",
+                "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+              }],
+              "detector": "$maxThreshold_2:THRESHOLD"
+            }],
+          "minContribution": 0.05,
+          "dimensions": ["D1", "D2"]
+        }]
+      },
+      {
+        "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+        "nested": [{
+          "grouper": "$Default grouper:MOCK_GROUPER",
+          "className": "org.apache.pinot.thirdeye.detection.wrapper.GrouperWrapper",
+          "nested": [{
+            "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+            "nested": [{
+              "nestedMetricUrns": ["thirdeye:metric:1"],
+              "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+              "nested": [{
+                "baselineValueProvider": "$maxThreshold_1:THRESHOLD",
+                "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+                "nested": [{
+                  "bucketPeriod": "P1D",
+                  "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+                }],
+                "detector": "$maxThreshold_1:THRESHOLD"
+              }]
+            }]
+          },
+            {
+              "className": "org.apache.pinot.thirdeye.detection.wrapper.ChildKeepingMergeWrapper",
+              "nested": [{
+                "nestedMetricUrns": ["thirdeye:metric:1"],
+                "className": "org.apache.pinot.thirdeye.detection.algorithm.DimensionWrapper",
+                "nested": [{
+                  "baselineValueProvider": "$maxThreshold_1:THRESHOLD",
+                  "className": "org.apache.pinot.thirdeye.detection.wrapper.BaselineFillingMergeWrapper",
+                  "nested": [{
+                    "bucketPeriod": "P1D",
+                    "className": "org.apache.pinot.thirdeye.detection.wrapper.AnomalyDetectorWrapper"
+                  }],
+                  "detector": "$maxThreshold_1:THRESHOLD"
+                }]
+              }]
+            }]
+        }]
+      }]
+    }]
+  },
+  "components": {
+    "rule1:THRESHOLD": {
+      "max": 100,
+      "className": "org.apache.pinot.thirdeye.detection.components.ThresholdRuleDetector"
+    }
+  },
+  "cron": "0 0 14 * * ? *"
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-4.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-4.yaml
new file mode 100644
index 0000000..72ecf8e
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-4.yaml
@@ -0,0 +1,26 @@
+detectionName: testPipeline
+description: My test pipeline
+type: COMPOSITE_ALERT
+cron: "0 0 14 * * ? *"
+alerts:
+  - type: METRIC_ALERT
+    name: metric alert on test_metric
+    metric: test_metric
+    dataset: test_dataset
+    filters:
+      D1:
+        - v1
+        - v2
+      D2:
+        - v3
+    dimensionExploration:
+      dimensions:
+        - D1
+        - D2
+      minContribution: 0.05
+    rules:
+      - detection:
+          - type: THRESHOLD
+            name: maxThreshold_1
+            params:
+              max: 100
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-5.yaml
similarity index 84%
copy from thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
copy to thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-5.yaml
index bba31ed..11faacb 100644
--- a/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/validators/entity-pipeline-config-2.yaml
+++ b/thirdeye/thirdeye-pinot/src/test/resources/org/apache/pinot/thirdeye/detection/yaml/translator/pipeline-config-5.yaml
@@ -1,10 +1,10 @@
 detectionName: testPipeline
 description: My test pipeline
 type: COMPOSITE_ALERT
-
+cron: "0 0 14 * * ? *"
 alerts:
   - type: METRIC_ALERT
-    name: metric alert on test_metric
+    name: metric_alert_on_test_metric
     metric: test_metric
     dataset: test_dataset
     filters:
@@ -38,13 +38,8 @@ alerts:
             name: maxThreshold_2
             params:
               max: 100
-        filter:
-          - type: THRESHOLD_RULE_FILTER
-            name: thresholdFilter_3
-            params:
-              min: 50
   - type: COMPOSITE_ALERT
-    name: composite alert on entity
+    name: composite_alert_on_entity
     alerts:
       - type: METRIC_ALERT
         name: metric alert on test_metric
@@ -57,7 +52,7 @@ alerts:
                 params:
                   max: 100
       - type: METRIC_ALERT
-        name: metric alert on test_metric
+        name: another_metric_alert_on_test_metric
         metric: test_metric
         dataset: test_dataset
         rules:


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