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 2020/07/24 21:36:35 UTC

[incubator-pinot] branch master updated: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example (#5713)

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 f5bb36b  [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example (#5713)
f5bb36b is described below

commit f5bb36b8e3b3a0ac01cfc09c9027098e3aa65552
Author: Akshay Rai <ak...@linkedin.com>
AuthorDate: Fri Jul 24 14:36:22 2020 -0700

    [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example (#5713)
    
    Given an anomaly, added an RCA highlights API which returns the cube results (and more). The PR also enables the notification pipeline to retrieve these results via a rest client. The results will be embedded into the alert email in a subsequent PR.
    
    Changes:
    * Created a generic Jersey client API (AbstractRestClient) for GET and POST requests and implemented an RCA rest client (ThirdEyeRcaRestClient) used for communicating with the RCA services.
    * Packaged the rest client under the commons package so that it can be leveraged by all the modules.
    * The Thirdeye rest client (admin service) account is configured in the detector.yaml configuration file.
    Ex:
    ```
    thirdeyeRestClient:
      adminUser: admin
      sessionKey: 1234
    ```
    * Introduced a constants package under commons and moved some of the RCA constants there.
    * Added unit tests for testing compose url and RCA rest client.
    * Tested the entire pipeline by deploying locally.
---
 .../anomaly/ThirdEyeAnomalyApplication.java        |  19 +++
 .../anomaly/ThirdEyeAnomalyConfiguration.java      |  10 ++
 .../rca/MultiDimensionalSummaryConstants.java      |  34 +++++
 .../constants/rca/RootCauseResourceConstants.java  |  29 ++++
 .../common/restclient/AbstractRestClient.java      | 166 +++++++++++++++++++++
 .../common/restclient/ParseResponseFunction.java   |  40 +++++
 .../pinot/thirdeye/common/restclient/Protocol.java |  35 +++++
 .../common/restclient/ThirdEyeRcaRestClient.java   | 109 ++++++++++++++
 .../ThirdEyeRestClientConfiguration.java           |  46 ++++++
 .../pinot/thirdeye/common/utils/SessionUtils.java  |  16 ++
 .../cube/entry/MultiDimensionalSummaryCLITool.java |  45 +++---
 .../dashboard/ThirdEyeDashboardApplication.java    |   2 +-
 .../dashboard/resources/SummaryResource.java       |  51 ++++---
 .../dashboard/resources/v2/RootCauseResource.java  |  54 ++++++-
 .../alert/scheme/DetectionAlertScheme.java         |   4 +
 .../alert/scheme/DetectionEmailAlerter.java        |   2 +-
 .../alert/scheme/DetectionJiraAlerter.java         |   3 +-
 .../content/BaseNotificationContent.java           |  12 +-
 .../content/templates/EntityGroupKeyContent.java   |   2 +-
 .../templates/HierarchicalAnomaliesContent.java    |   4 +-
 .../content/templates/MetricAnomaliesContent.java  |  59 ++++++--
 .../common/restclient/MockAbstractRestClient.java  |  52 +++++++
 .../restclient/MockThirdEyeRcaRestClient.java      |  56 +++++++
 .../common/restclient/TestAbstractRestClient.java  |  61 ++++++++
 .../restclient/TestThirdEyeRcaRestClient.java      |  65 ++++++++
 .../alert/scheme/DetectionEmailAlerterTest.java    |  15 ++
 .../alert/scheme/DetectionJiraAlerterTest.java     |  18 ++-
 .../templates/TestMetricAnomaliesContent.java      |  12 +-
 .../channels/TestJiraContentFormatter.java         |  17 ++-
 29 files changed, 956 insertions(+), 82 deletions(-)

diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyApplication.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyApplication.java
index 2a3686d..439b057 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyApplication.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyApplication.java
@@ -27,10 +27,13 @@ import org.apache.pinot.thirdeye.anomaly.events.HolidayEventsLoader;
 import org.apache.pinot.thirdeye.anomaly.events.MockEventsLoader;
 import org.apache.pinot.thirdeye.anomaly.monitor.MonitorJobScheduler;
 import org.apache.pinot.thirdeye.anomaly.task.TaskDriver;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRestClientConfiguration;
 import org.apache.pinot.thirdeye.common.time.TimeGranularity;
 import org.apache.pinot.thirdeye.auto.onboard.AutoOnboardService;
 import org.apache.pinot.thirdeye.common.BaseThirdEyeApplication;
 import org.apache.pinot.thirdeye.common.ThirdEyeSwaggerBundle;
+import org.apache.pinot.thirdeye.common.utils.SessionUtils;
+import org.apache.pinot.thirdeye.datalayer.dto.SessionDTO;
 import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
 import org.apache.pinot.thirdeye.datasource.pinot.resources.PinotDataSourceResource;
@@ -164,6 +167,10 @@ public class ThirdEyeAnomalyApplication
           modelDownloaderManager = new ModelDownloaderManager(config.getModelDownloaderConfig());
           modelDownloaderManager.start();
         }
+        if (config.getThirdEyeRestClientConfiguration() != null) {
+          ThirdEyeRestClientConfiguration restClientConfig = config.getThirdEyeRestClientConfiguration();
+          updateAdminSession(restClientConfig.getAdminUser(), restClientConfig.getSessionKey());
+        }
       }
 
       @Override
@@ -195,4 +202,16 @@ public class ThirdEyeAnomalyApplication
       }
     });
   }
+
+  private void updateAdminSession(String adminUser, String sessionKey) {
+    SessionDTO savedSession = DAO_REGISTRY.getSessionDAO().findBySessionKey(sessionKey);
+    long expiryMillis = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(365);
+    if (savedSession == null) {
+      SessionDTO sessionDTO = SessionUtils.buildServiceAccount(adminUser, sessionKey, expiryMillis);
+      DAO_REGISTRY.getSessionDAO().save(sessionDTO);
+    } else {
+      savedSession.setExpirationTime(expiryMillis);
+      DAO_REGISTRY.getSessionDAO().update(savedSession);
+    }
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyConfiguration.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyConfiguration.java
index 23d3b83..9203b1f 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyConfiguration.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/anomaly/ThirdEyeAnomalyConfiguration.java
@@ -25,6 +25,7 @@ import org.apache.pinot.thirdeye.anomaly.task.TaskDriverConfiguration;
 import org.apache.pinot.thirdeye.auto.onboard.AutoOnboardConfiguration;
 import org.apache.pinot.thirdeye.common.ThirdEyeConfiguration;
 import java.util.List;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRestClientConfiguration;
 
 
 public class ThirdEyeAnomalyConfiguration extends ThirdEyeConfiguration {
@@ -51,12 +52,21 @@ public class ThirdEyeAnomalyConfiguration extends ThirdEyeConfiguration {
   private MonitorConfiguration monitorConfiguration = new MonitorConfiguration();
   private AutoOnboardConfiguration autoOnboardConfiguration = new AutoOnboardConfiguration();
   private TaskDriverConfiguration taskDriverConfiguration = new TaskDriverConfiguration();
+  private ThirdEyeRestClientConfiguration teRestConfig = new ThirdEyeRestClientConfiguration();
   private DataAvailabilitySchedulingConfiguration
       dataAvailabilitySchedulingConfiguration = new DataAvailabilitySchedulingConfiguration();
   private String failureFromAddress;
   private String failureToAddress;
   private List<String> holidayCountriesWhitelist;
 
+  public ThirdEyeRestClientConfiguration getThirdEyeRestClientConfiguration() {
+    return teRestConfig;
+  }
+
+  public void setThirdEyeRestClientConfiguration(ThirdEyeRestClientConfiguration teRestConfig) {
+    this.teRestConfig = teRestConfig;
+  }
+
   public HolidayEventsLoaderConfiguration getHolidayEventsLoaderConfiguration() {
     return holidayEventsLoaderConfiguration;
   }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/MultiDimensionalSummaryConstants.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/MultiDimensionalSummaryConstants.java
new file mode 100644
index 0000000..a6bbd19
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/MultiDimensionalSummaryConstants.java
@@ -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.pinot.thirdeye.common.constants.rca;
+
+
+public class MultiDimensionalSummaryConstants {
+
+  // Cube algorithm parameters
+  public static final String CUBE_SUMMARY_SIZE = "summarySize";
+  public static final String CUBE_ONE_SIDE_ERROR = "oneSideError";
+  public static final String CUBE_DEPTH = "depth";
+  public static final String CUBE_DIM_HIERARCHIES = "hierarchies";
+  public static final String CUBE_ORDER_TYPE = "orderType";
+  public static final String CUBE_EXCLUDED_DIMENSIONS = "excludedDimensions";
+  public static final String CUBE_MANUAL_ORDER = "manualOrder";
+
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/RootCauseResourceConstants.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/RootCauseResourceConstants.java
new file mode 100644
index 0000000..2f2be86
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/constants/rca/RootCauseResourceConstants.java
@@ -0,0 +1,29 @@
+/*
+ * 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.common.constants.rca;
+
+public class RootCauseResourceConstants {
+  public static final String METRIC_URN = "metricUrn";
+  public static final String CURRENT_START = "currentStart";
+  public static final String CURRENT_END = "currentEnd";
+  public static final String BASELINE_START = "baselineStart";
+  public static final String BASELINE_END = "baselineEnd";
+  public static final String TIME_ZONE = "timeZone";
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
new file mode 100644
index 0000000..f195f8d
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
@@ -0,0 +1,166 @@
+/*
+ * 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.common.restclient;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import org.glassfish.jersey.client.ClientConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic Jersey Client API to perform GET and POST
+ */
+public abstract class AbstractRestClient {
+  private final Logger LOG = LoggerFactory.getLogger(this.getClass());
+
+  protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private final Client client;
+
+  /**
+   * Set up a default Jersey client.
+   * For unit tests, we use the alternate constructor to pass a mock.
+   */
+  public AbstractRestClient() {
+    this.client = ClientBuilder.newClient(new ClientConfig());
+  }
+
+  /**
+   * For testing only, create a client with an alternate client. This constructor allows
+   * unit tests to mock server communication.
+   */
+  public AbstractRestClient(Client client) {
+    this.client = client;
+  }
+
+  /**
+   * Perform a GET request to the given URL, accepts a method that will parse the response as a parameter.
+   * @param <T>  the type parameter defined as the return type of the response parser method
+   * @param url the http url
+   */
+  public <T> T doGet(String url, MultivaluedMap<String, Object> headers, ParseResponseFunction<Response, T> responseParserFunc) throws IOException {
+    long startedMillis = System.currentTimeMillis();
+
+    WebTarget webTarget = this.client.target(url);
+
+    Invocation.Builder invocationBuilder = webTarget.request(MediaType.APPLICATION_JSON).headers(headers);
+    Response response = invocationBuilder.get();
+
+    T result = responseParserFunc.parse(response);
+    long durationMillis = System.currentTimeMillis() - startedMillis;
+
+    LOG.info(String.format("GET '%s' succeeded in '%s'ms, response code '%s', response content '%s'.", url,
+        durationMillis, response.getStatus(), result));
+
+    return result;
+  }
+
+  /**
+   * Perform a POST request to the given URL, with a JSON or raw string as content.
+   * @param url the http url
+   * @param postContent the post content
+   */
+  public <T> T doPost(String url, MultivaluedMap<String, Object> headers, Object postContent,
+      ParseResponseFunction<Response, T> responseParserFunc) throws IOException {
+    long startedMillis = System.currentTimeMillis();
+
+    WebTarget webTarget = this.client.target(url);
+
+    Invocation.Builder invocationBuilder = webTarget.request(MediaType.APPLICATION_JSON).headers(headers);
+    Response response = invocationBuilder.post(Entity.entity(postContent, MediaType.APPLICATION_JSON));
+
+    T result = responseParserFunc.parse(response);
+    long durationMillis = System.currentTimeMillis() - startedMillis;
+
+    LOG.info(String.format("POST '%s' succeeded in '%s'ms, response code '%s', response content '%s'.", url,
+            durationMillis, response.getStatus(), result));
+
+    return result;
+  }
+
+  /**
+   * Composes a url from the host, api and queryParameters
+   */
+  protected String composeUrl(String host, String api, SortedMap<String, String> queryParameters) throws IOException {
+    return composeUrlGeneric(Protocol.HTTP, host, api, queryParameters);
+  }
+
+  /**
+   * Composes a url from the protocol, host, api and queryParameters
+   */
+  protected String composeUrlGeneric(Protocol protocol, String host, String api,
+      SortedMap<String, String> queryParameters) throws IOException {
+    StringBuilder url = new StringBuilder();
+    for (Protocol scheme : Protocol.values()) {
+      if (host.contains(scheme.val())) {
+        // protocol/scheme is already part of the host, do not append protocol
+        url.append(host);
+        break;
+      }
+    }
+    if (url.toString().isEmpty()) {
+      url.append(protocol.val()).append(host);
+    }
+    url.append(api);
+
+    if (queryParameters != null) {
+      try {
+        boolean firstQueryParam = true;
+        for (Map.Entry<String, String> entry : queryParameters.entrySet()) {
+          url.append(firstQueryParam ? "?" : "&");
+          firstQueryParam = false;
+          url.append(entry.getKey()).append('=');
+          url.append(URLEncoder.encode(entry.getValue(), "UTF-8")
+              .replace("+", "%20")
+              .replace("%2F", "/"));
+        }
+      } catch (UnsupportedEncodingException ex) {
+        // Should never happen since UTF-8 is always supported.
+        throw new IOException("Failed to URLEncode query parameters.", ex);
+      }
+    }
+    return url.toString();
+  }
+
+  /**
+   * Implementation for a single method class that can be used with doGet and doPost in order to parse a response stream
+   * to a Map<String,Object>
+   */
+  public class ResponseToMap implements ParseResponseFunction<Response, Map<String, Object>> {
+    public Map<String, Object> parse(Response response) {
+      return response.readEntity(new GenericType<HashMap<String, Object>>() { });
+    }
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ParseResponseFunction.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ParseResponseFunction.java
new file mode 100644
index 0000000..27be75b
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ParseResponseFunction.java
@@ -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.pinot.thirdeye.common.restclient;
+
+import java.io.IOException;
+
+
+/**
+ * A functional interface to allow passing a function to the rest client that would parse the response
+ *
+ * @param <T>  the type parameter
+ * @param <R>  the type parameter
+ */
+public interface ParseResponseFunction<T,R> {
+  /**
+   * Function that will take in a respnse and parse into an object
+   *
+   * @param t the t
+   * @return the r
+   * @throws IOException the io exception
+   */
+  R parse(T t) throws IOException;
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/Protocol.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/Protocol.java
new file mode 100644
index 0000000..7057dd2
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/Protocol.java
@@ -0,0 +1,35 @@
+/*
+ * 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.common.restclient;
+
+public enum Protocol {
+  HTTP("http://"),
+  HTTPS("https://");
+
+  private String val;
+
+  Protocol(String val) {
+    this.val = val;
+  }
+
+  public String val() {
+    return this.val;
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
new file mode 100644
index 0000000..2d09601
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
@@ -0,0 +1,109 @@
+/*
+ * 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.common.restclient;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.core.MultivaluedHashMap;
+import javax.ws.rs.core.MultivaluedMap;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.AuthResource;
+
+import static org.apache.pinot.thirdeye.common.constants.rca.MultiDimensionalSummaryConstants.*;
+import static org.apache.pinot.thirdeye.common.constants.rca.RootCauseResourceConstants.*;
+
+
+/**
+ * ThirdEye API Client to communicate with ThirdEye RCA services
+ */
+public class ThirdEyeRcaRestClient extends AbstractRestClient {
+
+  private final String DEFAULT_THIRDEYE_RCA_SERVICE_HOST = "localhost:1426";
+  private final String THIRDEYE_RCA_CUBE_URI = "/dashboard/summary/autoDimensionOrder";
+  private final String THIRDEYE_RCA_HIGHLIGHTS_URI = "/rootcause/highlights";
+
+  private ThirdEyePrincipal principal;
+  private String thirdEyeHost;
+
+  public ThirdEyeRcaRestClient(ThirdEyePrincipal principal, String host) {
+    super();
+    this.principal = principal;
+
+    if (StringUtils.isEmpty(host)) {
+      this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+    } else {
+      this.thirdEyeHost = host;
+    }
+  }
+
+  /** For testing only, create a client with an alternate Client. This constructor allows unit tests to mock server communication. */
+  /* package private */  ThirdEyeRcaRestClient(Client client, ThirdEyePrincipal principal) {
+    super(client);
+    this.principal = principal;
+    this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+  }
+
+  /**
+   * Fetch the likely root causes behind an anomaly
+   */
+  public Map<String, Object> getRootCauseHighlights(long anomalyId) throws IOException {
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put("anomalyId", String.valueOf(anomalyId));
+
+    MultivaluedMap<String, Object> headers = new MultivaluedHashMap<>();
+    headers.put("Cookie", Arrays.asList(AuthResource.AUTH_TOKEN_NAME + "=" + principal.getSessionKey()));
+
+    return doGet(
+        composeUrl(this.thirdEyeHost, THIRDEYE_RCA_HIGHLIGHTS_URI, queryParameters),
+        headers,
+        new ResponseToMap());
+  }
+
+  /**
+   * Retrieve the top dimension slices that explain the anomaly using the Cube Algorithm using wo1w
+   */
+  public Map<String, Object> getDimensionSummaryHighlights(String metricUrn, long startTime, long endTime,
+      long cubeDepth, long cubeSummarySize, boolean isOneSidedError) throws IOException {
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put(METRIC_URN, metricUrn);
+    queryParameters.put(BASELINE_START, String.valueOf(startTime - TimeUnit.DAYS.toMillis(7)));
+    queryParameters.put(BASELINE_END, String.valueOf(endTime - TimeUnit.DAYS.toMillis(7)));
+    queryParameters.put(CURRENT_START, String.valueOf(startTime));
+    queryParameters.put(CURRENT_END, String.valueOf(endTime));
+
+    queryParameters.put(CUBE_DEPTH, Long.toString(cubeDepth));
+    queryParameters.put(CUBE_ONE_SIDE_ERROR, Boolean.toString(isOneSidedError));
+    queryParameters.put(CUBE_SUMMARY_SIZE, Long.toString(cubeSummarySize));
+    queryParameters.put(CUBE_ORDER_TYPE, "auto");
+
+    MultivaluedMap<String, Object> headers = new MultivaluedHashMap<>();
+    headers.put("Cookie", Arrays.asList(AuthResource.AUTH_TOKEN_NAME + "=" + principal.getSessionKey()));
+
+    return doGet(
+        composeUrl(this.thirdEyeHost, THIRDEYE_RCA_CUBE_URI, queryParameters),
+        headers,
+        new ResponseToMap());
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRestClientConfiguration.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRestClientConfiguration.java
new file mode 100644
index 0000000..0558470
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRestClientConfiguration.java
@@ -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.pinot.thirdeye.common.restclient;
+
+import java.util.Optional;
+
+
+public class ThirdEyeRestClientConfiguration {
+  private String adminUser;
+  private String sessionKey;
+  private static final String DEFAULT_THIRDEYE_SERVICE_NAME = "thirdeye";
+  private static final String DEFAULT_THIRDEYE_SERVICE_KEY = "thirdeyeServiceKey";
+
+  public String getAdminUser() {
+    return Optional.ofNullable(adminUser).orElse(DEFAULT_THIRDEYE_SERVICE_NAME);
+  }
+
+  public void setAdminUser(String adminUser) {
+    this.adminUser = adminUser;
+  }
+
+  public String getSessionKey() {
+    return Optional.ofNullable(sessionKey).orElse(DEFAULT_THIRDEYE_SERVICE_KEY);
+  }
+
+  public void setSessionKey(String sessionKey) {
+    this.sessionKey = sessionKey;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/utils/SessionUtils.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/utils/SessionUtils.java
new file mode 100644
index 0000000..23f622a
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/utils/SessionUtils.java
@@ -0,0 +1,16 @@
+package org.apache.pinot.thirdeye.common.utils;
+
+import org.apache.pinot.thirdeye.datalayer.dto.SessionDTO;
+import org.apache.pinot.thirdeye.datalayer.pojo.SessionBean;
+
+
+public class SessionUtils {
+  public static SessionDTO buildServiceAccount(String user, String sessionKey, long expiryInMillis) {
+    SessionDTO sessionDTO = new SessionDTO();
+    sessionDTO.setPrincipal(user);
+    sessionDTO.setPrincipalType(SessionBean.PrincipalType.SERVICE);
+    sessionDTO.setSessionKey(sessionKey);
+    sessionDTO.setExpirationTime(expiryInMillis);
+    return sessionDTO;
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/entry/MultiDimensionalSummaryCLITool.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/entry/MultiDimensionalSummaryCLITool.java
index 5366910..e6785f9 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/entry/MultiDimensionalSummaryCLITool.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/cube/entry/MultiDimensionalSummaryCLITool.java
@@ -25,11 +25,9 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Multimap;
 import org.apache.pinot.thirdeye.cube.additive.AdditiveDBClient;
-import org.apache.pinot.thirdeye.cube.additive.AdditiveRow;
 import org.apache.pinot.thirdeye.cube.data.dbrow.Dimensions;
 import org.apache.pinot.thirdeye.cube.cost.BalancedCostFunction;
 import org.apache.pinot.thirdeye.cube.cost.CostFunction;
-import org.apache.pinot.thirdeye.cube.data.dbclient.CubePinotClient;
 import org.apache.pinot.thirdeye.dashboard.Utils;
 import org.apache.pinot.thirdeye.cube.summary.SummaryResponse;
 import org.apache.pinot.thirdeye.datasource.ThirdEyeCacheRegistry;
@@ -55,6 +53,9 @@ import org.joda.time.DateTimeZone;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.thirdeye.common.constants.rca.MultiDimensionalSummaryConstants.*;
+import static org.apache.pinot.thirdeye.common.constants.rca.RootCauseResourceConstants.*;
+
 
 public class MultiDimensionalSummaryCLITool {
   private static final Logger LOG = LoggerFactory.getLogger(MultiDimensionalSummaryCLITool.class);
@@ -78,7 +79,7 @@ public class MultiDimensionalSummaryCLITool {
     options.addOption(dimensions);
 
     Option excludedDimensions =
-        Option.builder("notDim").longOpt("excludedDimensions").desc("dimension names to be excluded").hasArg()
+        Option.builder("notDim").longOpt(CUBE_EXCLUDED_DIMENSIONS).desc("dimension names to be excluded").hasArg()
             .argName("LIST").build();
     options.addOption(excludedDimensions);
 
@@ -87,45 +88,45 @@ public class MultiDimensionalSummaryCLITool {
     options.addOption(filters);
 
     Option currentStart =
-        Option.builder("cstart").longOpt("currentStart").desc("current start time inclusive").hasArg().argName("MILLIS")
+        Option.builder("cstart").longOpt(CURRENT_START).desc("current start time inclusive").hasArg().argName("MILLIS")
             .required().build();
     options.addOption(currentStart);
 
     Option currentEnd =
-        Option.builder("cend").longOpt("currentEnd").desc("current end time exclusive").hasArg().argName("MILLIS")
+        Option.builder("cend").longOpt(CURRENT_END).desc("current end time exclusive").hasArg().argName("MILLIS")
             .required().build();
     options.addOption(currentEnd);
 
     Option baselineStart =
-        Option.builder("bstart").longOpt("baselineStart").desc("baseline start time inclusive").hasArg()
+        Option.builder("bstart").longOpt(BASELINE_START).desc("baseline start time inclusive").hasArg()
             .argName("MILLIS").required().build();
     options.addOption(baselineStart);
 
     Option baselineEnd =
-        Option.builder("bend").longOpt("baselineEnd").desc("baseline end time exclusive").hasArg().argName("MILLIS")
+        Option.builder("bend").longOpt(BASELINE_END).desc("baseline end time exclusive").hasArg().argName("MILLIS")
             .required().build();
     options.addOption(baselineEnd);
 
     Option size =
-        Option.builder("size").longOpt("summarySize").desc("size of summary").hasArg().argName("NUMBER").build();
+        Option.builder("size").longOpt(CUBE_SUMMARY_SIZE).desc("size of summary").hasArg().argName("NUMBER").build();
     options.addOption(size);
 
-    Option depth = Option.builder("depth").desc("number of top dimensions").hasArg().argName("NUMBER").build();
+    Option depth = Option.builder(CUBE_DEPTH).desc("number of top dimensions").hasArg().argName("NUMBER").build();
     options.addOption(depth);
 
-    Option hierarchies = Option.builder("h").longOpt("hierarchies")
+    Option hierarchies = Option.builder("h").longOpt(CUBE_DIM_HIERARCHIES)
         .desc("dimension hierarchies (a list of lists in Json format)").hasArg().argName("JSON")
         .build();
     options.addOption(hierarchies);
 
-    Option oneSideError = Option.builder("oneSideError").desc("enable one side error summary").build();
+    Option oneSideError = Option.builder(CUBE_ONE_SIDE_ERROR).desc("enable one side error summary").build();
     options.addOption(oneSideError);
 
-    Option manualOrder = Option.builder("manualOrder").desc("use manual dimension order").build();
+    Option manualOrder = Option.builder(CUBE_MANUAL_ORDER).desc("use manual dimension order").build();
     options.addOption(manualOrder);
 
     Option dateTimeZone =
-        Option.builder("timeZone").desc("time zone id in Joda library").hasArg().argName("ID").build();
+        Option.builder(TIME_ZONE).desc("time zone id in Joda library").hasArg().argName("ID").build();
     options.addOption(dateTimeZone);
 
     Option costFunctionClass = Option.builder("cost").longOpt("costFunction").desc(
@@ -198,21 +199,21 @@ public class MultiDimensionalSummaryCLITool {
       String dataset = commandLine.getOptionValue("dataset");
       String metricName = commandLine.getOptionValue("metric");
       String dimensionString = commandLine.getOptionValue("dimensionString", "");
-      String excludedDimensionString = commandLine.getOptionValue("excludedDimensions", "");
+      String excludedDimensionString = commandLine.getOptionValue(CUBE_EXCLUDED_DIMENSIONS, "");
       String filterJson = commandLine.getOptionValue("filters", "{}");
-      long currentStart = Long.parseLong(commandLine.getOptionValue("currentStart"));
-      long currentEnd = Long.parseLong(commandLine.getOptionValue("currentEnd"));
-      long baselineStart = Long.parseLong(commandLine.getOptionValue("baselineStart"));
-      long baselineEnd = Long.parseLong(commandLine.getOptionValue("baselineEnd"));
+      long currentStart = Long.parseLong(commandLine.getOptionValue(CURRENT_START));
+      long currentEnd = Long.parseLong(commandLine.getOptionValue(CURRENT_END));
+      long baselineStart = Long.parseLong(commandLine.getOptionValue(BASELINE_START));
+      long baselineEnd = Long.parseLong(commandLine.getOptionValue(BASELINE_END));
 
       int summarySize = Integer.parseInt(commandLine.getOptionValue("size", "10"));
-      int depth = Integer.parseInt(commandLine.getOptionValue("depth", "3"));
-      String hierarchiesJson = commandLine.getOptionValue("hierarchies", "[]");
+      int depth = Integer.parseInt(commandLine.getOptionValue(CUBE_DEPTH, "3"));
+      String hierarchiesJson = commandLine.getOptionValue(CUBE_DIM_HIERARCHIES, "[]");
       if (commandLine.hasOption("manual")) {
         depth = 0;
       }
-      boolean oneSideError = commandLine.hasOption("oneSideError");
-      String dateTimeZoneId = commandLine.getOptionValue("timeZone", DateTimeZone.UTC.getID());
+      boolean oneSideError = commandLine.hasOption(CUBE_ONE_SIDE_ERROR);
+      String dateTimeZoneId = commandLine.getOptionValue(TIME_ZONE, DateTimeZone.UTC.getID());
       DateTimeZone timeZone = DateTimeZone.forID(dateTimeZoneId);
 
       // Create cost function
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 c80a4a5..c3decbc 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
@@ -238,8 +238,8 @@ public class ThirdEyeDashboardApplication
       }
 
       env.jersey().register(new ThirdEyeAuthFilter(authenticator, authConfig.getAllowedPaths(), authConfig.getAdminUsers(), DAORegistry.getInstance().getSessionDAO()));
-      env.jersey().register(new AuthResource(authenticator, authConfig.getCookieTTL() * 1000));
       env.jersey().register(new AuthValueFactoryProvider.Binder<>(ThirdEyePrincipal.class));
+      env.jersey().register(new AuthResource(authenticator, authConfig.getCookieTTL() * 1000));
     }
 
     if (config.getModelDownloaderConfig() != null) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
index df28684..c6f1d54 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/SummaryResource.java
@@ -59,6 +59,9 @@ import org.joda.time.DateTimeZone;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.thirdeye.common.constants.rca.MultiDimensionalSummaryConstants.*;
+import static org.apache.pinot.thirdeye.common.constants.rca.RootCauseResourceConstants.*;
+
 
 @Path(value = "/dashboard")
 public class SummaryResource {
@@ -66,11 +69,11 @@ public class SummaryResource {
   private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   private static final ThirdEyeCacheRegistry CACHE_REGISTRY_INSTANCE = ThirdEyeCacheRegistry.getInstance();
 
-  private static final String DEFAULT_TIMEZONE_ID = "UTC";
-  private static final String DEFAULT_DEPTH = "3";
-  private static final String DEFAULT_HIERARCHIES = "[]";
-  private static final String DEFAULT_ONE_SIDE_ERROR = "false";
-  private static final String DEFAULT_EXCLUDED_DIMENSIONS = "";
+  public static final String DEFAULT_TIMEZONE_ID = "UTC";
+  public static final String DEFAULT_DEPTH = "3";
+  public static final String DEFAULT_HIERARCHIES = "[]";
+  public static final String DEFAULT_ONE_SIDE_ERROR = "false";
+  public static final String DEFAULT_EXCLUDED_DIMENSIONS = "";
   private static final String JAVASCRIPT_NULL_STRING = "undefined";
   private static final String HTML_STRING_ENCODING = "UTF-8";
 
@@ -96,21 +99,21 @@ public class SummaryResource {
   @Path(value = "/summary/autoDimensionOrder")
   @Produces(MediaType.APPLICATION_JSON)
   public String buildSummary(
-      @QueryParam("metricUrn") String metricUrn,
+      @QueryParam(METRIC_URN) String metricUrn,
       @QueryParam("dataset") String dataset,
       @QueryParam("metric") String metric,
-      @QueryParam("currentStart") long currentStartInclusive,
-      @QueryParam("currentEnd") long currentEndExclusive,
-      @QueryParam("baselineStart") long baselineStartInclusive,
-      @QueryParam("baselineEnd") long baselineEndExclusive,
+      @QueryParam(CURRENT_START) long currentStartInclusive,
+      @QueryParam(CURRENT_END) long currentEndExclusive,
+      @QueryParam(BASELINE_START) long baselineStartInclusive,
+      @QueryParam(BASELINE_END) long baselineEndExclusive,
       @QueryParam("dimensions") String groupByDimensions,
       @QueryParam("filters") String filterJsonPayload,
-      @QueryParam("summarySize") int summarySize,
-      @QueryParam("depth") @DefaultValue(DEFAULT_DEPTH) int depth,
-      @QueryParam("hierarchies") @DefaultValue(DEFAULT_HIERARCHIES) String hierarchiesPayload,
-      @QueryParam("oneSideError") @DefaultValue(DEFAULT_ONE_SIDE_ERROR) boolean doOneSideError,
-      @QueryParam("excludedDimensions") @DefaultValue(DEFAULT_EXCLUDED_DIMENSIONS) String excludedDimensions,
-      @QueryParam("timeZone") @DefaultValue(DEFAULT_TIMEZONE_ID) String timeZone) throws Exception {
+      @QueryParam(CUBE_SUMMARY_SIZE) int summarySize,
+      @QueryParam(CUBE_DEPTH) @DefaultValue(DEFAULT_DEPTH) int depth,
+      @QueryParam(CUBE_DIM_HIERARCHIES) @DefaultValue(DEFAULT_HIERARCHIES) String hierarchiesPayload,
+      @QueryParam(CUBE_ONE_SIDE_ERROR) @DefaultValue(DEFAULT_ONE_SIDE_ERROR) boolean doOneSideError,
+      @QueryParam(CUBE_EXCLUDED_DIMENSIONS) @DefaultValue(DEFAULT_EXCLUDED_DIMENSIONS) String excludedDimensions,
+      @QueryParam(TIME_ZONE) @DefaultValue(DEFAULT_TIMEZONE_ID) String timeZone) throws Exception {
     if (summarySize < 1) summarySize = 1;
 
     String metricName = metric;
@@ -176,18 +179,18 @@ public class SummaryResource {
   @Path(value = "/summary/manualDimensionOrder")
   @Produces(MediaType.APPLICATION_JSON)
   public String buildSummaryManualDimensionOrder(
-      @QueryParam("metricUrn") String metricUrn,
+      @QueryParam(METRIC_URN) String metricUrn,
       @QueryParam("dataset") String dataset,
       @QueryParam("metric") String metric,
-      @QueryParam("currentStart") long currentStartInclusive,
-      @QueryParam("currentEnd") long currentEndExclusive,
-      @QueryParam("baselineStart") long baselineStartInclusive,
-      @QueryParam("baselineEnd") long baselineEndExclusive,
+      @QueryParam(CURRENT_START) long currentStartInclusive,
+      @QueryParam(CURRENT_END) long currentEndExclusive,
+      @QueryParam(BASELINE_START) long baselineStartInclusive,
+      @QueryParam(BASELINE_END) long baselineEndExclusive,
       @QueryParam("dimensions") String groupByDimensions,
       @QueryParam("filters") String filterJsonPayload,
-      @QueryParam("summarySize") int summarySize,
-      @QueryParam("oneSideError") @DefaultValue(DEFAULT_ONE_SIDE_ERROR) boolean doOneSideError,
-      @QueryParam("timeZone") @DefaultValue(DEFAULT_TIMEZONE_ID) String timeZone) throws Exception {
+      @QueryParam(CUBE_SUMMARY_SIZE) int summarySize,
+      @QueryParam(CUBE_ONE_SIDE_ERROR) @DefaultValue(DEFAULT_ONE_SIDE_ERROR) boolean doOneSideError,
+      @QueryParam(TIME_ZONE) @DefaultValue(DEFAULT_TIMEZONE_ID) String timeZone) throws Exception {
     if (summarySize < 1) summarySize = 1;
 
     String metricName = metric;
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java
index 8a9f7dc..345f2fa 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java
@@ -19,19 +19,14 @@
 
 package org.apache.pinot.thirdeye.dashboard.resources.v2;
 
-import org.apache.pinot.thirdeye.api.Constants;
-import org.apache.pinot.thirdeye.dashboard.resources.v2.pojo.RootCauseEntity;
-import org.apache.pinot.thirdeye.rootcause.Entity;
-import org.apache.pinot.thirdeye.rootcause.RCAFramework;
-import org.apache.pinot.thirdeye.rootcause.RCAFrameworkExecutionResult;
-import org.apache.pinot.thirdeye.rootcause.util.EntityUtils;
-import org.apache.pinot.thirdeye.rootcause.impl.TimeRangeEntity;
+import com.google.common.base.Preconditions;
+import io.dropwizard.auth.Auth;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -42,9 +37,23 @@ import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
+import joptsimple.internal.Strings;
+import org.apache.pinot.thirdeye.api.Constants;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.SummaryResource;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.pojo.RootCauseEntity;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.apache.pinot.thirdeye.rootcause.Entity;
+import org.apache.pinot.thirdeye.rootcause.RCAFramework;
+import org.apache.pinot.thirdeye.rootcause.RCAFrameworkExecutionResult;
+import org.apache.pinot.thirdeye.rootcause.impl.TimeRangeEntity;
+import org.apache.pinot.thirdeye.rootcause.util.EntityUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.thirdeye.dashboard.resources.SummaryResource.*;
+
 
 @Path(value = "/rootcause")
 @Produces(MediaType.APPLICATION_JSON)
@@ -57,13 +66,42 @@ public class RootCauseResource {
   private static final long ANALYSIS_RANGE_MAX = TimeUnit.DAYS.toMillis(32);
   private static final long ANOMALY_RANGE_MAX = TimeUnit.DAYS.toMillis(32);
   private static final long BASELINE_RANGE_MAX = ANOMALY_RANGE_MAX;
+  private static final int DEFAULT_HIGHLIGHT_CUBE_SUMMARY_SIZE = 3;
+  private static final int DEFAULT_HIGHLIGHT_CUBE_DEPTH = 3;
 
   private final List<RootCauseEntityFormatter> formatters;
   private final Map<String, RCAFramework> frameworks;
+  private final SummaryResource summaryResource;
 
   public RootCauseResource(Map<String, RCAFramework> frameworks, List<RootCauseEntityFormatter> formatters) {
     this.frameworks = frameworks;
     this.formatters = formatters;
+    this.summaryResource = new SummaryResource();
+  }
+
+  @GET
+  @Path("/highlights")
+  @ApiOperation("Retrieve the likely root causes behind an anomaly")
+  public Map<String, Object> highlights(
+      @ApiParam(value = "internal id of the anomaly")
+      @QueryParam("anomalyId") long anomalyId,
+      @Auth ThirdEyePrincipal principal) throws Exception {
+    Map<String, Object> responseMessage = new HashMap<>();
+    MergedAnomalyResultDTO anomalyDTO = DAORegistry.getInstance().getMergedAnomalyResultDAO().findById(anomalyId);
+    Preconditions.checkNotNull(anomalyDTO, "Anomaly doesn't exist in ThirdEye's repository");
+
+    // In the highlights api we retrieve only the top 3 results across 3 dimensions.
+    // TODO: polish the results to make it more meaningful
+    String cubeHighlights = this.summaryResource.buildSummary(
+        anomalyDTO.getMetricUrn(), anomalyDTO.getCollection(), anomalyDTO.getMetric(),
+        anomalyDTO.getStartTime(), anomalyDTO.getEndTime(),
+        anomalyDTO.getStartTime() - TimeUnit.DAYS.toMillis(7),
+        anomalyDTO.getEndTime() - TimeUnit.DAYS.toMillis(7), Strings.EMPTY, Strings.EMPTY,
+        DEFAULT_HIGHLIGHT_CUBE_SUMMARY_SIZE, DEFAULT_HIGHLIGHT_CUBE_DEPTH, SummaryResource.DEFAULT_HIERARCHIES,
+        false, DEFAULT_EXCLUDED_DIMENSIONS, SummaryResource.DEFAULT_TIMEZONE_ID);
+    responseMessage.put("cubeResults", cubeHighlights);
+
+    return responseMessage;
   }
 
   @GET
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionAlertScheme.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionAlertScheme.java
index c89fc2b..e5f015a 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionAlertScheme.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionAlertScheme.java
@@ -98,4 +98,8 @@ public abstract class DetectionAlertScheme {
           numOfAnomalies, this.subsConfig.getId(), e);
     }
   }
+
+  protected BaseNotificationContent getNotificationContent(Properties alertSchemeClientConfigs) {
+    return buildNotificationContent(alertSchemeClientConfigs);
+  }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerter.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerter.java
index 11fee55..fab7c67 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerter.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerter.java
@@ -148,7 +148,7 @@ public class DetectionEmailAlerter extends DetectionAlertScheme {
     blacklistRecipients(recipients);
     validateAlert(recipients, anomalies);
 
-    BaseNotificationContent content = buildNotificationContent(emailClientConfigs);
+    BaseNotificationContent content = getNotificationContent(emailClientConfigs);
     EmailEntity emailEntity = new EmailContentFormatter(emailClientConfigs, content, this.teConfig, subsConfig)
         .getEmailEntity(anomalies);
     if (Strings.isNullOrEmpty(this.subsConfig.getFrom())) {
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerter.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerter.java
index 4d2c7bc..16ff525 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerter.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerter.java
@@ -21,7 +21,6 @@ package org.apache.pinot.thirdeye.detection.alert.scheme;
 
 import com.atlassian.jira.rest.client.api.domain.Issue;
 import com.google.common.base.Preconditions;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -142,7 +141,7 @@ public class DetectionJiraAlerter extends DetectionAlertScheme {
     List<AnomalyResult> anomalyResultListOfGroup = new ArrayList<>(anomalies);
     anomalyResultListOfGroup.sort(COMPARATOR_DESC);
 
-    BaseNotificationContent content = super.buildNotificationContent(jiraClientConfig);
+    BaseNotificationContent content = getNotificationContent(jiraClientConfig);
 
     return new JiraContentFormatter(this.jiraAdminConfig, jiraClientConfig, content, this.teConfig, subsetSubsConfig)
         .getJiraEntity(notification.getDimensionFilters(), anomalyResultListOfGroup);
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/BaseNotificationContent.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/BaseNotificationContent.java
index 20a66b6..0528d61 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/BaseNotificationContent.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/BaseNotificationContent.java
@@ -525,11 +525,12 @@ public abstract class BaseNotificationContent implements NotificationContent {
     String entityName;
     String anomalyType;
     String properties;
+    String metricUrn;
 
     public AnomalyReportEntity(String anomalyId, String anomalyURL, String baselineVal, String currentVal, String lift,
         boolean positiveLift, Double swi, List<String> dimensions, String duration, String feedback, String function,
         String funcDescription, String metric, String startTime, String endTime, String timezone, String issueType,
-        String anomalyType, String properties) {
+        String anomalyType, String properties, String metricUrn) {
       this.anomalyId = anomalyId;
       this.anomalyURL = anomalyURL;
       this.baselineVal = baselineVal;
@@ -556,6 +557,7 @@ public abstract class BaseNotificationContent implements NotificationContent {
       this.issueType = issueType;
       this.anomalyType = anomalyType;
       this.properties = properties;
+      this.metricUrn = metricUrn;
     }
 
     public void setSeasonalValues(COMPARE_MODE compareMode, double seasonalValue, double current) {
@@ -872,5 +874,13 @@ public abstract class BaseNotificationContent implements NotificationContent {
     public void setWo4wLift(String wo4wLift) {
       this.wo4wLift = wo4wLift;
     }
+
+    public String getMetricUrn() {
+      return metricUrn;
+    }
+
+    public void setMetricUrn(String metricUrn) {
+      this.metricUrn = metricUrn;
+    }
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/EntityGroupKeyContent.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/EntityGroupKeyContent.java
index ea7a8e7..34ffcb4 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/EntityGroupKeyContent.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/EntityGroupKeyContent.java
@@ -161,7 +161,7 @@ public class EntityGroupKeyContent extends BaseNotificationContent {
         detectionConfig.getName(), detectionConfig.getDescription(), anomaly.getMetric(),
         getDateString(anomaly.getStartTime(), dateTimeZone), getDateString(anomaly.getEndTime(), dateTimeZone),
         getTimezoneString(dateTimeZone), getIssueType(anomaly), anomaly.getType().getLabel(),
-        ThirdEyeStringUtils.encodeCompactedProperties(props));
+        ThirdEyeStringUtils.encodeCompactedProperties(props), anomaly.getMetricUrn());
 
     // Extract out the whitelisted metrics
     if (anomaly.getProperties() != null && anomaly.getProperties().containsKey(PROP_SUB_ENTITY_NAME)
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/HierarchicalAnomaliesContent.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/HierarchicalAnomaliesContent.java
index 9a63c4c..499be0b 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/HierarchicalAnomaliesContent.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/HierarchicalAnomaliesContent.java
@@ -169,8 +169,8 @@ public class HierarchicalAnomaliesContent extends BaseNotificationContent {
         getTimezoneString(dateTimeZone),
         getIssueType(anomaly),
         anomaly.getType().getLabel(),
-        ThirdEyeStringUtils.encodeCompactedProperties(props)
-
+        ThirdEyeStringUtils.encodeCompactedProperties(props),
+        anomaly.getMetricUrn()
     );
 
     List<String> affectedCountries = getMatchedFilterValues(anomaly, "country");
diff --git a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/MetricAnomaliesContent.java b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/MetricAnomaliesContent.java
index 08ab966..81cb685 100644
--- a/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/MetricAnomaliesContent.java
+++ b/thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/notification/content/templates/MetricAnomaliesContent.java
@@ -23,28 +23,29 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
-import org.apache.pinot.thirdeye.datalayer.util.ThirdEyeStringUtils;
-import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.anomaly.ThirdEyeAnomalyConfiguration;
 import org.apache.pinot.thirdeye.anomaly.alert.util.AlertScreenshotHelper;
 import org.apache.pinot.thirdeye.anomalydetection.context.AnomalyFeedback;
 import org.apache.pinot.thirdeye.anomalydetection.context.AnomalyResult;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRcaRestClient;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
+import org.apache.pinot.thirdeye.datalayer.dto.DetectionAlertConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.DetectionConfigDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.EventDTO;
 import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.pinot.thirdeye.datalayer.util.ThirdEyeStringUtils;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
 import org.apache.pinot.thirdeye.notification.content.BaseNotificationContent;
-import org.apache.pinot.thirdeye.rootcause.impl.MetricEntity;
-import org.apache.pinot.thirdeye.util.ThirdEyeUtils;
 import org.joda.time.DateTime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -57,13 +58,27 @@ public class MetricAnomaliesContent extends BaseNotificationContent {
   private static final Logger LOG = LoggerFactory.getLogger(MetricAnomaliesContent.class);
 
   private DetectionConfigManager configDAO = null;
+  private ThirdEyeRcaRestClient rcaClient;
+
+  public MetricAnomaliesContent() {
+  }
 
-  public MetricAnomaliesContent() {}
+  // For testing
+  public MetricAnomaliesContent(ThirdEyeRcaRestClient rcaClient) {
+    this.rcaClient = rcaClient;
+  }
 
   @Override
   public void init(Properties properties, ThirdEyeAnomalyConfiguration configuration) {
     super.init(properties, configuration);
     this.configDAO = DAORegistry.getInstance().getDetectionConfigManager();
+
+    if (this.rcaClient == null) {
+      ThirdEyePrincipal principal = new ThirdEyePrincipal();
+      principal.setName(this.thirdEyeAnomalyConfig.getThirdEyeRestClientConfiguration().getAdminUser());
+      principal.setSessionKey(this.thirdEyeAnomalyConfig.getThirdEyeRestClientConfiguration().getSessionKey());
+      this.rcaClient = new ThirdEyeRcaRestClient(principal, this.thirdEyeAnomalyConfig.getDashboardHost());
+    }
   }
 
   @Override
@@ -148,7 +163,8 @@ public class MetricAnomaliesContent extends BaseNotificationContent {
           getTimezoneString(dateTimeZone),
           getIssueType(anomaly),
           anomaly.getType().getLabel(),
-          ThirdEyeStringUtils.encodeCompactedProperties(props)
+          ThirdEyeStringUtils.encodeCompactedProperties(props),
+          anomaly.getMetricUrn()
       );
 
       // dimension filters / values
@@ -199,6 +215,21 @@ public class MetricAnomaliesContent extends BaseNotificationContent {
     templateData.put("metricToAnomalyDetailsMap", metricAnomalyReports.asMap());
     templateData.put("functionToId", functionToId);
 
+    // Display RCA highlights in email only if report contains anomalies belonging to a single metric.
+    // Note: Once we have a sophisticated rca highlight support and users start seeing value, we'll
+    // enable it for all the metrics.
+    if (metricAnomalyReports.keySet().size() == 1) {
+      String anomalyId = metricAnomalyReports.values().iterator().next().getAnomalyId();
+      Map<String, Object> rcaHighlights = new HashMap<>();
+      try {
+        rcaHighlights = this.rcaClient.getRootCauseHighlights(Long.parseLong(anomalyId));
+      } catch (IOException e) {
+        LOG.error("Failed to retrieve the RCA Highlights for anomaly " + anomalyId, e);
+      }
+      LOG.info("Setting rootCauseHighlights in email template " + rcaHighlights);
+      templateData.put("rootCauseHighlights", rcaHighlights);
+    }
+
     return templateData;
   }
 }
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockAbstractRestClient.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockAbstractRestClient.java
new file mode 100644
index 0000000..e59e3f4
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockAbstractRestClient.java
@@ -0,0 +1,52 @@
+/*
+ * 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.common.restclient;
+
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.Response;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.*;
+
+
+public class MockAbstractRestClient {
+
+  public static Client setupMockClient(final Map<String, Object> expectedResponse) {
+    Client client = mock(Client.class);
+
+    WebTarget webTarget = mock(WebTarget.class);
+    when(client.target(anyString())).thenReturn(webTarget);
+
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    when(webTarget.request(anyString())).thenReturn(builder);
+    when(builder.headers(any())).thenReturn(builder);
+
+    Response response = mock(Response.class);
+    when(builder.get()).thenReturn(response);
+    when(response.readEntity(any(GenericType.class))).thenReturn(expectedResponse);
+
+    return client;
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockThirdEyeRcaRestClient.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockThirdEyeRcaRestClient.java
new file mode 100644
index 0000000..a9877f2
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/MockThirdEyeRcaRestClient.java
@@ -0,0 +1,56 @@
+/*
+ * 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.common.restclient;
+
+import java.io.IOException;
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Invocation;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.GenericType;
+import javax.ws.rs.core.Response;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+
+public class MockThirdEyeRcaRestClient {
+
+  public static ThirdEyeRcaRestClient setupMockClient(final Map<String, Object> expectedResponse) {
+    Client client = mock(Client.class);
+
+    WebTarget webTarget = mock(WebTarget.class);
+    when(client.target(anyString())).thenReturn(webTarget);
+
+    Invocation.Builder builder = mock(Invocation.Builder.class);
+    when(webTarget.request(anyString())).thenReturn(builder);
+    when(builder.headers(any())).thenReturn(builder);
+
+    Response response = mock(Response.class);
+    when(builder.get()).thenReturn(response);
+    when(response.readEntity(any(GenericType.class))).thenReturn(expectedResponse);
+
+    ThirdEyePrincipal principal = new ThirdEyePrincipal();
+    principal.setSessionKey("dummy");
+
+    return new ThirdEyeRcaRestClient(client, principal);
+  }
+}
\ No newline at end of file
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestAbstractRestClient.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestAbstractRestClient.java
new file mode 100644
index 0000000..cd2e311
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestAbstractRestClient.java
@@ -0,0 +1,61 @@
+/*
+ * 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.common.restclient;
+
+import java.io.IOException;
+import java.util.TreeMap;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class TestAbstractRestClient extends AbstractRestClient {
+
+  /**
+   * Test compose url with null query parameters creates a valid url.
+   */
+  @Test
+  public void testComposeUrlNullQueryParameters() throws IOException {
+    String api = "/api/my/api";
+    String host = "host";
+
+    String actualUrl = composeUrl(host, api, null);
+    String expectedUrl = String.format("http://%s%s", host, api);
+
+    Assert.assertEquals(actualUrl,expectedUrl);
+  }
+
+  /**
+   * Test compose url parameter with space create a valid url
+   */
+  @Test
+  public void testComposeUrlGenericParameterWithSpaceAndSlash() throws IOException{
+    String api = "/api/my/api";
+    String host = "host";
+    String parameterName = "parameter";
+    String parameterValue = "param value";
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put(parameterName,parameterValue);
+
+    String actualUrl = composeUrlGeneric(Protocol.HTTPS, host, api, queryParameters);
+    String expectedUrl = "https://host/api/my/api?parameter=param%20value";
+
+    Assert.assertEquals(actualUrl,expectedUrl);
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestThirdEyeRcaRestClient.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestThirdEyeRcaRestClient.java
new file mode 100644
index 0000000..80f757e
--- /dev/null
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/common/restclient/TestThirdEyeRcaRestClient.java
@@ -0,0 +1,65 @@
+/*
+ * 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.common.restclient;
+
+import java.util.HashMap;
+import java.util.Map;
+import javax.ws.rs.client.Client;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+
+public class TestThirdEyeRcaRestClient {
+
+  private DAOTestBase testDAOProvider;
+  private long anomalyId;
+
+  @BeforeMethod
+  public void beforeMethod() throws Exception {
+    this.testDAOProvider = DAOTestBase.getInstance();
+    DAORegistry daoRegistry = DAORegistry.getInstance();
+
+    MergedAnomalyResultDTO anomaly = new MergedAnomalyResultDTO();
+    anomaly.setCollection("collection");
+    anomaly.setMetric("metric");
+    anomalyId = daoRegistry.getMergedAnomalyResultDAO().save(anomaly);
+  }
+
+  @Test
+  public void testGetAllHighlights() throws Exception {
+    Map<String, Object> expectedResponse = new HashMap<>();
+    expectedResponse.put("cubeResults", "{}");
+
+    Client client = MockAbstractRestClient.setupMockClient(expectedResponse);
+
+    ThirdEyePrincipal principal = new ThirdEyePrincipal();
+    principal.setSessionKey("dummy");
+    ThirdEyeRcaRestClient rcaClient = new ThirdEyeRcaRestClient(client, principal);
+    Map<String, Object> result = rcaClient.getRootCauseHighlights(anomalyId);
+
+    Assert.assertTrue(result.containsKey("cubeResults"));
+    Assert.assertEquals(expectedResponse.get("cubeResults"), result.get("cubeResults"));
+  }
+}
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerterTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerterTest.java
index d17b3db..f882b79 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerterTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionEmailAlerterTest.java
@@ -20,9 +20,12 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import org.apache.commons.mail.HtmlEmail;
 import org.apache.pinot.thirdeye.anomaly.ThirdEyeAnomalyConfiguration;
+import org.apache.pinot.thirdeye.common.restclient.MockThirdEyeRcaRestClient;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRcaRestClient;
 import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
@@ -36,6 +39,8 @@ import org.apache.pinot.thirdeye.detection.alert.DetectionAlertFilterNotificatio
 import org.apache.pinot.thirdeye.detection.alert.DetectionAlertFilterResult;
 import org.apache.pinot.thirdeye.detection.alert.filter.SubscriptionUtils;
 import org.apache.pinot.thirdeye.notification.commons.EmailEntity;
+import org.apache.pinot.thirdeye.notification.content.BaseNotificationContent;
+import org.apache.pinot.thirdeye.notification.content.templates.MetricAnomaliesContent;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -148,11 +153,21 @@ public class DetectionEmailAlerterTest {
     final HtmlEmail htmlEmail = mock(HtmlEmail.class);
     when(htmlEmail.send()).thenReturn("sent");
 
+    Map<String, Object> expectedResponse = new HashMap<>();
+    expectedResponse.put("cubeResults", "{}");
+    ThirdEyeRcaRestClient rcaClient = MockThirdEyeRcaRestClient.setupMockClient(expectedResponse);
+    MetricAnomaliesContent metricAnomaliesContent = new MetricAnomaliesContent(rcaClient);
+
     DetectionEmailAlerter emailAlerter = new DetectionEmailAlerter(this.alertConfigDTO, this.thirdEyeConfig, notificationResults) {
       @Override
       protected HtmlEmail getHtmlContent(EmailEntity emailEntity) {
         return htmlEmail;
       }
+
+      @Override
+      protected BaseNotificationContent getNotificationContent(Properties emailClientConfigs) {
+        return metricAnomaliesContent;
+      }
     };
     // Executes successfully without errors
     emailAlerter.run();
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerterTest.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerterTest.java
index 77f87be..7e98604 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerterTest.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/detection/alert/scheme/DetectionJiraAlerterTest.java
@@ -22,8 +22,11 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import org.apache.pinot.thirdeye.anomaly.ThirdEyeAnomalyConfiguration;
+import org.apache.pinot.thirdeye.common.restclient.MockThirdEyeRcaRestClient;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRcaRestClient;
 import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionAlertConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
@@ -38,6 +41,8 @@ import org.apache.pinot.thirdeye.detection.alert.DetectionAlertFilterResult;
 import org.apache.pinot.thirdeye.detection.alert.filter.SubscriptionUtils;
 import org.apache.pinot.thirdeye.notification.commons.JiraEntity;
 import org.apache.pinot.thirdeye.notification.commons.ThirdEyeJiraClient;
+import org.apache.pinot.thirdeye.notification.content.BaseNotificationContent;
+import org.apache.pinot.thirdeye.notification.content.templates.MetricAnomaliesContent;
 import org.apache.pinot.thirdeye.notification.formatter.channels.TestJiraContentFormatter;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeMethod;
@@ -140,7 +145,18 @@ public class DetectionJiraAlerterTest {
     doNothing().when(jiraClient).updateIssue(any(Issue.class), any(JiraEntity.class));
     doNothing().when(jiraClient).addComment(any(Issue.class), anyString());
 
-    DetectionJiraAlerter jiraAlerter = new DetectionJiraAlerter(this.alertConfigDTO, this.thirdEyeConfig, notificationResults, jiraClient);
+    Map<String, Object> expectedResponse = new HashMap<>();
+    expectedResponse.put("cubeResults", "{}");
+    ThirdEyeRcaRestClient rcaClient = MockThirdEyeRcaRestClient.setupMockClient(expectedResponse);
+    MetricAnomaliesContent metricAnomaliesContent = new MetricAnomaliesContent(rcaClient);
+
+    DetectionJiraAlerter jiraAlerter = new DetectionJiraAlerter(this.alertConfigDTO, this.thirdEyeConfig,
+        notificationResults, jiraClient) {
+      @Override
+      protected BaseNotificationContent getNotificationContent(Properties emailClientConfigs) {
+        return metricAnomaliesContent;
+      }
+    };
 
     // Executes successfully without errors
     jiraAlerter.run();
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/content/templates/TestMetricAnomaliesContent.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/content/templates/TestMetricAnomaliesContent.java
index 60d0d23..b993f05 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/content/templates/TestMetricAnomaliesContent.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/content/templates/TestMetricAnomaliesContent.java
@@ -17,7 +17,12 @@
 package org.apache.pinot.thirdeye.notification.content.templates;
 
 import java.util.Properties;
+import javax.ws.rs.client.Client;
 import org.apache.pinot.thirdeye.anomaly.AnomalyType;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.common.restclient.MockAbstractRestClient;
+import org.apache.pinot.thirdeye.common.restclient.MockThirdEyeRcaRestClient;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRcaRestClient;
 import org.apache.pinot.thirdeye.constant.AnomalyResultSource;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
 import org.apache.pinot.thirdeye.datalayer.bao.DetectionConfigManager;
@@ -66,6 +71,7 @@ import org.testng.annotations.Test;
 
 import static org.apache.pinot.thirdeye.datalayer.DaoTestUtils.*;
 import static org.apache.pinot.thirdeye.notification.commons.SmtpConfiguration.*;
+import static org.mockito.Mockito.*;
 
 
 public class TestMetricAnomaliesContent {
@@ -220,8 +226,12 @@ public class TestMetricAnomaliesContent {
     metric.setAlias(TEST + "::" + TEST);
     metricDAO.save(metric);
 
+    Map<String, Object> expectedResponse = new HashMap<>();
+    expectedResponse.put("cubeResults", "{}");
+    ThirdEyeRcaRestClient rcaClient = MockThirdEyeRcaRestClient.setupMockClient(expectedResponse);
+    MetricAnomaliesContent metricAnomaliesContent = new MetricAnomaliesContent(rcaClient);
     EmailContentFormatter
-        contentFormatter = new EmailContentFormatter(new Properties(), new MetricAnomaliesContent(),
+        contentFormatter = new EmailContentFormatter(new Properties(), metricAnomaliesContent,
         thirdeyeAnomalyConfig, DaoTestUtils.getTestNotificationConfig("Test Config"));
     EmailEntity emailEntity = contentFormatter.getEmailEntity(anomalies);
 
diff --git a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/formatter/channels/TestJiraContentFormatter.java b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/formatter/channels/TestJiraContentFormatter.java
index a5311c0..20aac96 100644
--- a/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/formatter/channels/TestJiraContentFormatter.java
+++ b/thirdeye/thirdeye-pinot/src/test/java/org/apache/pinot/thirdeye/notification/formatter/channels/TestJiraContentFormatter.java
@@ -32,6 +32,8 @@ import org.apache.commons.io.IOUtils;
 import org.apache.pinot.thirdeye.anomaly.AnomalyType;
 import org.apache.pinot.thirdeye.anomaly.ThirdEyeAnomalyConfiguration;
 import org.apache.pinot.thirdeye.anomalydetection.context.AnomalyResult;
+import org.apache.pinot.thirdeye.common.restclient.MockThirdEyeRcaRestClient;
+import org.apache.pinot.thirdeye.common.restclient.ThirdEyeRcaRestClient;
 import org.apache.pinot.thirdeye.constant.AnomalyResultSource;
 import org.apache.pinot.thirdeye.datalayer.bao.DAOTestBase;
 import org.apache.pinot.thirdeye.datalayer.bao.DatasetConfigManager;
@@ -50,6 +52,7 @@ import org.apache.pinot.thirdeye.detection.alert.scheme.DetectionAlertScheme;
 import org.apache.pinot.thirdeye.notification.commons.JiraConfiguration;
 import org.apache.pinot.thirdeye.notification.commons.JiraEntity;
 import org.apache.pinot.thirdeye.notification.content.BaseNotificationContent;
+import org.apache.pinot.thirdeye.notification.content.templates.MetricAnomaliesContent;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.testng.Assert;
@@ -86,6 +89,7 @@ public class TestJiraContentFormatter {
   private Long subsId2;
   private DAOTestBase testDAOProvider;
   private List<AnomalyResult> anomalies;
+  private MetricAnomaliesContent metricAnomaliesContent;
 
   @BeforeMethod
   public void beforeMethod() throws Exception {
@@ -148,6 +152,11 @@ public class TestJiraContentFormatter {
     this.alertConfigDimAlerter = createDimRecipientsDetectionAlertConfig(this.detectionConfigId);
     subsId2 = this.alertConfigDAO.save(this.alertConfigDimAlerter);
     alertConfigDimAlerter.setId(subsId2);
+
+    Map<String, Object> expectedResponse = new HashMap<>();
+    expectedResponse.put("cubeResults", "{}");
+    ThirdEyeRcaRestClient rcaClient = MockThirdEyeRcaRestClient.setupMockClient(expectedResponse);
+    this.metricAnomaliesContent = new MetricAnomaliesContent(rcaClient);
   }
 
   @AfterClass(alwaysRun = true)
@@ -248,9 +257,9 @@ public class TestJiraContentFormatter {
     jiraClientConfig.put(PROP_LABELS, Arrays.asList("test-label-1", "test-label-2"));
     jiraClientConfig.put(PROP_SUBJECT_STYLE, AlertConfigBean.SubjectType.METRICS);
 
-    BaseNotificationContent content = DetectionAlertScheme.buildNotificationContent(jiraClientConfig);
     JiraContentFormatter jiraContent = new JiraContentFormatter(
-        JiraConfiguration.createFromProperties(jiraConfiguration), jiraClientConfig, content, teConfig, this.alertConfigDTO);
+        JiraConfiguration.createFromProperties(jiraConfiguration), jiraClientConfig, this.metricAnomaliesContent, teConfig,
+        this.alertConfigDTO);
 
     JiraEntity jiraEntity = jiraContent.getJiraEntity(ArrayListMultimap.create(), this.anomalies);
 
@@ -288,9 +297,9 @@ public class TestJiraContentFormatter {
     jiraClientConfig.put(PROP_LABELS, Arrays.asList("test-label-1", "test-label-2"));
     jiraClientConfig.put(PROP_SUBJECT_STYLE, AlertConfigBean.SubjectType.METRICS);
 
-    BaseNotificationContent content = DetectionAlertScheme.buildNotificationContent(jiraClientConfig);
     JiraContentFormatter jiraContent = new JiraContentFormatter(
-        JiraConfiguration.createFromProperties(jiraConfiguration), jiraClientConfig, content, teConfig, this.alertConfigDimAlerter);
+        JiraConfiguration.createFromProperties(jiraConfiguration), jiraClientConfig, this.metricAnomaliesContent,
+        teConfig, this.alertConfigDimAlerter);
 
     Multimap<String, String> dimensionKeys1 = ArrayListMultimap.create();
     dimensionKeys1.put("key", "value");


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