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/01/17 00:48:43 UTC

[incubator-pinot] branch master updated: [TE] Translate Alert Config to Yaml (#3702)

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 c48a38b  [TE] Translate Alert Config to Yaml (#3702)
c48a38b is described below

commit c48a38b7cdeff645b8fe85ca05663700ce08edd8
Author: Akshay Rai <ak...@gmail.com>
AuthorDate: Wed Jan 16 16:48:39 2019 -0800

    [TE] Translate Alert Config to Yaml (#3702)
---
 .../dashboard/ThirdEyeDashboardApplication.java    |   5 +-
 .../detection/DetectionMigrationResource.java      |  64 +++++-------
 .../detection/LegacyAnomalyFunctionTranslator.java |   1 +
 .../registry/DetectionAlertRegistry.java           |   8 +-
 .../yaml/YamlDetectionAlertConfigTranslator.java   |  41 ++++----
 .../detection/DetectionMigrationResourceTest.java  | 111 +++++++++++++++++++++
 6 files changed, 165 insertions(+), 65 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
index 196c3dd..141bd01 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/ThirdEyeDashboardApplication.java
@@ -177,9 +177,8 @@ public class ThirdEyeDashboardApplication
     env.jersey().register(new ThirdEyeResource());
     env.jersey().register(new DataResource(anomalyFunctionFactory, alertFilterFactory));
     env.jersey().register(new AnomaliesResource(anomalyFunctionFactory, alertFilterFactory));
-    env.jersey().register(new DetectionMigrationResource(
-        DAO_REGISTRY.getMetricConfigDAO(), DAO_REGISTRY.getAnomalyFunctionDAO(),
-        DAO_REGISTRY.getDetectionConfigManager(), DAO_REGISTRY.getDatasetConfigDAO(), anomalyFunctionFactory, alertFilterFactory));
+    env.jersey().register(new DetectionMigrationResource(DAO_REGISTRY.getAnomalyFunctionDAO(),
+        DAO_REGISTRY.getDetectionConfigManager(), DAO_REGISTRY.getDatasetConfigDAO()));
     env.jersey().register(new OnboardResource(config));
     env.jersey().register(new EntityMappingResource());
     env.jersey().register(new OnboardDatasetMetricResource());
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResource.java
index bc544a4..7aec064 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResource.java
@@ -26,25 +26,21 @@ import org.apache.pinot.thirdeye.datalayer.bao.AnomalyFunctionManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.MetricConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.AlertConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.AnomalyFunctionDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DatasetConfigDTO;
-import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.detector.email.filter.AlertFilterFactory;
 import org.apache.pinot.thirdeye.detector.function.AnomalyFunctionFactory;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import javax.ws.rs.GET;
-import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Response;
 import org.apache.commons.lang3.StringUtils;
 import org.joda.time.Period;
 import org.slf4j.Logger;
@@ -53,6 +49,7 @@ import org.yaml.snakeyaml.DumperOptions;
 import org.yaml.snakeyaml.Yaml;
 
 import static org.apache.pinot.thirdeye.anomaly.merge.AnomalyMergeStrategy.*;
+import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
 
 
 /**
@@ -66,7 +63,6 @@ public class DetectionMigrationResource {
   private static final String PROP_WINDOW_SIZE = "windowSize";
   private static final String PROP_WINDOW_UNIT = "windowUnit";
 
-  private final LegacyAnomalyFunctionTranslator translator;
   private final AnomalyFunctionManager anomalyFunctionDAO;
   private final DetectionConfigManager detectionConfigDAO;
   private final DatasetConfigManager datasetConfigDAO;
@@ -74,18 +70,13 @@ public class DetectionMigrationResource {
 
   /**
    * Instantiates a new Detection migration resource.
-   *
-   * @param anomalyFunctionFactory the anomaly function factory
    */
-  public DetectionMigrationResource(MetricConfigManager metricConfigDAO, AnomalyFunctionManager anomalyFunctionDAO,
+  public DetectionMigrationResource(AnomalyFunctionManager anomalyFunctionDAO,
       DetectionConfigManager detectionConfigDAO,
-      DatasetConfigManager datasetConfigDAO,
-      AnomalyFunctionFactory anomalyFunctionFactory,
-      AlertFilterFactory alertFilterFactory) {
+      DatasetConfigManager datasetConfigDAO) {
     this.anomalyFunctionDAO = anomalyFunctionDAO;
     this.detectionConfigDAO = detectionConfigDAO;
     this.datasetConfigDAO = datasetConfigDAO;
-    this.translator = new LegacyAnomalyFunctionTranslator(metricConfigDAO, anomalyFunctionFactory, alertFilterFactory);
     DumperOptions options = new DumperOptions();
     options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
     options.setPrettyFlow(true);
@@ -279,35 +270,30 @@ public class DetectionMigrationResource {
     return detectorYaml;
   }
 
-  /**
-   * This endpoint takes in a anomaly function Id and translate the anomaly function config to a
-   * detection config of the new pipeline and then persist it in to database.
-   *
-   * @param anomalyFunctionId the anomaly function id
-   * @return the response
-   * @throws Exception the exception
-   */
-  @POST
-  public Response migrateToDetectionPipeline(@QueryParam("id") long anomalyFunctionId, @QueryParam("name") String name,
-      @QueryParam("lastTimestamp") Long lastTimestamp) throws Exception {
-    AnomalyFunctionDTO anomalyFunctionDTO = this.anomalyFunctionDAO.findById(anomalyFunctionId);
-    DetectionConfigDTO config = this.translator.translate(anomalyFunctionDTO);
+  Map<String, Object> translateAlertToYaml(AlertConfigDTO alertConfigDTO) {
+    Map<String, Object> yamlConfigs = new LinkedHashMap<>();
 
-    if (!StringUtils.isBlank(name)) {
-      config.setName(name);
-    }
+    yamlConfigs.put(PROP_SUBS_GROUP_NAME, alertConfigDTO.getName());
+    yamlConfigs.put(PROP_CRON, alertConfigDTO.getCronExpression());
+    yamlConfigs.put(PROP_ACTIVE, alertConfigDTO.isActive());
+    yamlConfigs.put(PROP_APPLICATION, alertConfigDTO.getApplication());
+    yamlConfigs.put(PROP_EMAIL_SUBJECT_TYPE, alertConfigDTO.getSubjectType());
+    yamlConfigs.put(PROP_FROM, alertConfigDTO.getFromAddress());
 
-    config.setLastTimestamp(System.currentTimeMillis());
-    if (lastTimestamp != null) {
-      config.setLastTimestamp(lastTimestamp);
-    }
+    yamlConfigs.put(PROP_TYPE, "DEFAULT_ALERTER_PIPELINE");
 
-    this.detectionConfigDAO.save(config);
-    if (config.getId() == null) {
-      throw new WebApplicationException(String.format("Could not migrate anomaly function %d", anomalyFunctionId));
-    }
+    Map<String, Object> recipients = new LinkedHashMap<>();
+    recipients.put("to", alertConfigDTO.getReceiverAddresses().getTo());
+    recipients.put("cc", alertConfigDTO.getReceiverAddresses().getCc());
+    recipients.put("bcc", alertConfigDTO.getReceiverAddresses().getBcc());
+    yamlConfigs.put(PROP_RECIPIENTS, recipients);
+
+    Map<String, Object> alertSchemes = new LinkedHashMap<>();
+    alertSchemes.put(PROP_TYPE, "EMAIL");
+    yamlConfigs.put(PROP_ALERT_SCHEMES, alertSchemes);
+
+    yamlConfigs.put(PROP_DETECTION_CONFIG_IDS, alertConfigDTO.getEmailConfig().getFunctionIds());
 
-    LOGGER.info("Created detection config {} for anomaly function {}", config.getId(), anomalyFunctionDTO.getId());
-    return Response.ok(config.getId()).build();
+    return yamlConfigs;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/LegacyAnomalyFunctionTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/LegacyAnomalyFunctionTranslator.java
index 050328d..7b2905e 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/LegacyAnomalyFunctionTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/LegacyAnomalyFunctionTranslator.java
@@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
 /**
  * The Legacy Anomaly function translator.
  */
+@Deprecated
 public class LegacyAnomalyFunctionTranslator {
   private static final Logger LOGGER = LoggerFactory.getLogger(LegacyAnomalyFunctionTranslator.class);
 
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionAlertRegistry.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionAlertRegistry.java
index fa1cd98..89a9380 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionAlertRegistry.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/annotation/registry/DetectionAlertRegistry.java
@@ -126,15 +126,15 @@ public class DetectionAlertRegistry {
    * Look up the {@link #ALERT_SCHEME_MAP} for the Alert scheme class name from the type
    */
   public String lookupAlertSchemes(String schemeType) {
-    Preconditions.checkArgument(ALERT_SCHEME_MAP.containsKey(schemeType), schemeType + " not found in registry");
-    return ALERT_SCHEME_MAP.get(schemeType);
+    Preconditions.checkArgument(ALERT_SCHEME_MAP.containsKey(schemeType.toUpperCase()), schemeType + " not found in registry");
+    return ALERT_SCHEME_MAP.get(schemeType.toUpperCase());
   }
 
   /**
    * Look up the {@link #ALERT_SUPPRESSOR_MAP} for the Alert suppressor class name from the type
    */
   public String lookupAlertSuppressors(String suppressorType) {
-    Preconditions.checkArgument(ALERT_SUPPRESSOR_MAP.containsKey(suppressorType), suppressorType + " not found in registry");
-    return ALERT_SUPPRESSOR_MAP.get(suppressorType);
+    Preconditions.checkArgument(ALERT_SUPPRESSOR_MAP.containsKey(suppressorType.toUpperCase()), suppressorType + " not found in registry");
+    return ALERT_SUPPRESSOR_MAP.get(suppressorType.toUpperCase());
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
index 1cfa75c..06b02c1 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/yaml/YamlDetectionAlertConfigTranslator.java
@@ -44,25 +44,25 @@ public class YamlDetectionAlertConfigTranslator {
   public static final String PROP_DETECTION_CONFIG_IDS = "detectionConfigIds";
   public static final String PROP_RECIPIENTS = "recipients";
 
-  static final String PROP_SUBS_GROUP_NAME = "subscriptionGroupName";
-  static final String PROP_CRON = "cron";
-  static final String PROP_ACTIVE = "active";
-  static final String PROP_APPLICATION = "application";
-  static final String PROP_FROM = "fromAddress";
-  static final String PROP_ONLY_FETCH_LEGACY_ANOMALIES = "onlyFetchLegacyAnomalies";
-  static final String PROP_EMAIL_SUBJECT_TYPE = "emailSubjectStyle";
-  static final String PROP_ALERT_SCHEMES = "alertSchemes";
-  static final String PROP_ALERT_SUPPRESSORS = "alertSuppressors";
-  static final String PROP_REFERENCE_LINKS = "referenceLinks";
-
-  static final String PROP_TYPE = "type";
-  static final String PROP_CLASS_NAME = "className";
-  static final String PROP_PARAM = "params";
-
-  static final String PROP_DIMENSION = "dimension";
-  static final String PROP_DIMENSION_RECIPIENTS = "dimensionRecipients";
-  static final String PROP_TIME_WINDOWS = "timeWindows";
-  static final String CRON_SCHEDULE_DEFAULT = "0 0/5 * * * ? *"; // Every 5 min
+  public static final String PROP_SUBS_GROUP_NAME = "subscriptionGroupName";
+  public static final String PROP_CRON = "cron";
+  public static final String PROP_ACTIVE = "active";
+  public static final String PROP_APPLICATION = "application";
+  public static final String PROP_FROM = "fromAddress";
+  public static final String PROP_ONLY_FETCH_LEGACY_ANOMALIES = "onlyFetchLegacyAnomalies";
+  public static final String PROP_EMAIL_SUBJECT_TYPE = "emailSubjectStyle";
+  public static final String PROP_ALERT_SCHEMES = "alertSchemes";
+  public static final String PROP_ALERT_SUPPRESSORS = "alertSuppressors";
+  public static final String PROP_REFERENCE_LINKS = "referenceLinks";
+
+  public static final String PROP_TYPE = "type";
+  public static final String PROP_CLASS_NAME = "className";
+  public static final String PROP_PARAM = "params";
+
+  public static final String PROP_DIMENSION = "dimension";
+  public static final String PROP_DIMENSION_RECIPIENTS = "dimensionRecipients";
+  public static final String PROP_TIME_WINDOWS = "timeWindows";
+  public static final String CRON_SCHEDULE_DEFAULT = "0 0/5 * * * ? *"; // Every 5 min
 
   private static final DetectionAlertRegistry DETECTION_ALERT_REGISTRY = DetectionAlertRegistry.getInstance();
   private static final Set<String> PROPERTY_KEYS = new HashSet<>(
@@ -192,7 +192,10 @@ public class YamlDetectionAlertConfigTranslator {
 
     alertConfigDTO.setCronExpression(MapUtils.getString(yamlAlertConfig, PROP_CRON, CRON_SCHEDULE_DEFAULT));
     alertConfigDTO.setActive(MapUtils.getBooleanValue(yamlAlertConfig, PROP_ACTIVE, true));
+
+    // TODO: Remove all references to onlyFetchLegacyAnomalies after migration
     alertConfigDTO.setOnlyFetchLegacyAnomalies(MapUtils.getBooleanValue(yamlAlertConfig, PROP_ONLY_FETCH_LEGACY_ANOMALIES, false));
+
     alertConfigDTO.setSubjectType((AlertConfigBean.SubjectType) MapUtils.getObject(yamlAlertConfig, PROP_EMAIL_SUBJECT_TYPE, AlertConfigBean.SubjectType.METRICS));
 
     Map<String, String> refLinks = MapUtils.getMap(yamlAlertConfig, PROP_REFERENCE_LINKS);
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResourceTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResourceTest.java
new file mode 100644
index 0000000..9073043
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/DetectionMigrationResourceTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.commons.collections.MapUtils;
+import org.apache.pinot.thirdeye.datalayer.bao.AlertConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.AnomalyFunctionManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
+import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.bao.MergedAnomalyResultManager;
+import org.apache.pinot.thirdeye.datalayer.dto.AlertConfigDTO;
+import org.apache.pinot.thirdeye.datalayer.pojo.AlertConfigBean;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.apache.pinot.thirdeye.detection.alert.DetectionAlertFilterRecipients;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.apache.pinot.thirdeye.detection.yaml.YamlDetectionAlertConfigTranslator.*;
+
+
+public class DetectionMigrationResourceTest {
+
+  private DAOTestBase testDAOProvider;
+  private DatasetConfigManager datasetDAO;
+  private MergedAnomalyResultManager anomalyDAO;
+  private AnomalyFunctionManager anomalyFunctionDAO;
+  private AlertConfigManager alertConfigDAO;
+  private DetectionConfigManager detectionConfigDAO;
+  private DetectionAlertConfigManager detectionAlertConfigDAO;
+
+  private DetectionMigrationResource migrationResource;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    this.testDAOProvider = DAOTestBase.getInstance();
+    this.datasetDAO = DAORegistry.getInstance().getDatasetConfigDAO();
+    this.anomalyDAO = DAORegistry.getInstance().getMergedAnomalyResultDAO();
+    this.anomalyFunctionDAO = DAORegistry.getInstance().getAnomalyFunctionDAO();
+    this.alertConfigDAO = DAORegistry.getInstance().getAlertConfigDAO();
+    this.detectionConfigDAO = DAORegistry.getInstance().getDetectionConfigManager();
+    this.detectionAlertConfigDAO = DAORegistry.getInstance().getDetectionAlertConfigManager();
+
+    migrationResource = new DetectionMigrationResource(anomalyFunctionDAO, detectionConfigDAO, datasetDAO);
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void afterMethod() {
+    this.testDAOProvider.cleanup();
+  }
+
+  @Test
+  public void testTranslateAlertToYaml() throws Exception {
+    AlertConfigDTO alertConfigDTO = new AlertConfigDTO();
+    alertConfigDTO.setActive(true);
+    alertConfigDTO.setName("test_notification");
+    alertConfigDTO.setApplication("test_application");
+    alertConfigDTO.setSubjectType(AlertConfigBean.SubjectType.ALERT);
+    alertConfigDTO.setCronExpression("0 0 14 * * ? *");
+    alertConfigDTO.setFromAddress("test@thirdeye.com");
+    alertConfigDTO.setReceiverAddresses(new DetectionAlertFilterRecipients(
+        Collections.singleton("to@test"),
+        Collections.singleton("cc@test"),
+        Collections.singleton("bcc@test")));
+
+    AlertConfigBean.EmailConfig emailConfig = new AlertConfigBean.EmailConfig();
+    emailConfig.setFunctionIds(Arrays.asList(1l, 2l, 3l, 4l));
+    emailConfig.setAnomalyWatermark(9999);
+    alertConfigDTO.setEmailConfig(emailConfig);
+
+    Map<String, Object> yamlMap = migrationResource.translateAlertToYaml(alertConfigDTO);
+
+    Assert.assertTrue(MapUtils.getBoolean(yamlMap, "active"));
+    Assert.assertEquals(yamlMap.get(PROP_SUBS_GROUP_NAME), "test_notification");
+    Assert.assertEquals(yamlMap.get(PROP_CRON), "0 0 14 * * ? *");
+    Assert.assertEquals(yamlMap.get(PROP_APPLICATION), "test_application");
+    Assert.assertEquals(yamlMap.get(PROP_EMAIL_SUBJECT_TYPE), AlertConfigBean.SubjectType.ALERT);
+    Assert.assertEquals(yamlMap.get(PROP_FROM), "test@thirdeye.com");
+    Assert.assertEquals(yamlMap.get(PROP_TYPE), "DEFAULT_ALERTER_PIPELINE");
+    Assert.assertEquals(yamlMap.get(PROP_DETECTION_CONFIG_IDS), Arrays.asList(1l, 2l, 3l, 4l));
+    Assert.assertNotNull(yamlMap.get(PROP_ALERT_SCHEMES));
+    Assert.assertEquals(((Map<String, Object>) yamlMap.get(PROP_ALERT_SCHEMES)).get(PROP_TYPE), "EMAIL");
+    Assert.assertNotNull(yamlMap.get(PROP_RECIPIENTS));
+    Assert.assertEquals(((Map<String, Object>) yamlMap.get(PROP_RECIPIENTS)).get("to"), Collections.singleton("to@test"));
+    Assert.assertEquals(((Map<String, Object>) yamlMap.get(PROP_RECIPIENTS)).get("cc"), Collections.singleton("cc@test"));
+    Assert.assertEquals(((Map<String, Object>) yamlMap.get(PROP_RECIPIENTS)).get("bcc"), Collections.singleton("bcc@test"));
+  }
+}
\ No newline at end of file


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