You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/17 22:57:25 UTC

[GitHub] [incubator-pinot] akshayrai opened a new pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

akshayrai opened a new pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713


   Changes:
   * Created a generic API rest client (`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.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] akshayrai merged pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

Posted by GitBox <gi...@apache.org>.
akshayrai merged pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] akshayrai commented on a change in pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

Posted by GitBox <gi...@apache.org>.
akshayrai commented on a change in pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713#discussion_r459815532



##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {
+  private final Logger LOG = LoggerFactory.getLogger(this.getClass());
+
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private final HttpURLConnectionFactory connectionFactory;
+
+  /**
+   * Using a URLFactory to create URLs allows unit tests to mock out server communication.
+   */
+  public class HttpURLConnectionFactory {
+    public HttpURLConnection openConnection(String url) throws MalformedURLException, IOException {
+      return (HttpURLConnection) new URL(url).openConnection();
+    }
+  }
+
+  /**
+   * Set up the client with a default URLFactory that creates real HTTP connections.
+   * For unit tests, we use the alternate constructor to pass a mock.
+   */
+  public AbstractRestClient() {
+    connectionFactory = new HttpURLConnectionFactory();
+  }
+
+  /**
+   * For testing only, create a client with an alternate URLFactory. This constructor allows
+   * unit tests to mock server communication.
+   */
+  public AbstractRestClient(HttpURLConnectionFactory connectionFactory) {
+    this.connectionFactory = connectionFactory;
+  }
+
+  /**
+   * Perform a GET request to the given URL, accepts a method that will parse the response as a parameter.
+   * A timeout of zero is interpreted as an infinite timeout.
+   * @param <T>  the type parameter defined as the return type of the response parser method
+   * @param url the http url
+   * @param host the host to connect to
+   * @param headers the headers for communication
+   */
+  public <T> T doGet(String url, String host, Map<String, String> headers,
+      ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    return doGet(url, host, headers, 0, 0, responseParserFunc);
+  }
+
+  /**
+   * 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
+   * @param host the host to connect to
+   * @param connectTimeout timeout in milliseconds
+   * @param readTimeout timeout in milliseconds
+   */
+  public <T> T doGet(String url, String host, Map<String, String> reqHeaders, int connectTimeout, int readTimeout,
+      ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    Map<String, String> headers = new HashMap<>(reqHeaders);
+    headers.put("User-Agent", getClass().getName());
+    headers.put("Host", host);
+    headers.put("Accept", "application/json");
+
+    return doRequest(url, "GET", new byte[0], null, connectTimeout, readTimeout, headers,
+        responseParserFunc);
+  }
+
+  /**
+   * Perform a POST request to the given URL, with a JSON or raw string as content.
+   * If the payload is an object that will be serialized to JSON the isContentTypeJSON must be set to true
+   * @param <T>  the type parameter defined as the return type of the response parser method
+   * @param url the http url
+   * @param host the host to connect to
+   * @param postContent the post content
+   * @param isContentTypeJson flag indicating if the content is json type or not
+   */
+  public <T> T doPost(String url, String host, Map<String, String> reqHeaders, Object postContent,
+      boolean isContentTypeJson, ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    OBJECT_MAPPER.writeValue(baos, postContent);
+    byte[] content = baos.toByteArray();
+    String contentType = isContentTypeJson ? "application/json" : null;
+
+    Map<String, String> headers = new HashMap<>(reqHeaders);
+    headers.put("User-Agent", getClass().getName());
+    headers.put("Host", host);
+    headers.put("Accept", "application/json");
+
+    return doRequest(url, "POST", content, contentType, 0, 0, headers,
+        responseParserFunc);
+  }
+
+  /**
+   * Send a request to the given URL with the given parameters.
+   * @param <T> the type parameter defined as the return type of the response parser method
+   * @param url server url.
+   * @param method HTTP method.
+   * @param content request content.
+   * @param contentType request content type.
+   * @param connectTimeoutMillis connection timeout.
+   * @param readTimeoutMillis read timeout.
+   * @param headers any additional request headers.
+   * @return T the result of the parser functions applied to the response content.
+   */
+  private <T> T doRequest(String url, String method, byte[] content, String contentType, int connectTimeoutMillis,
+      int readTimeoutMillis, Map<String, String> headers, ParseResponseFunction<InputStream, T> responseParserFunc)
+      throws IOException {
+    long startedMillis = System.currentTimeMillis();
+
+    HttpURLConnection conn = this.connectionFactory.openConnection(url);

Review comment:
       no longer applies

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {

Review comment:
       Good point! I have updated to use the Jersey Client API




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] vincentchenjl commented on a change in pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

Posted by GitBox <gi...@apache.org>.
vincentchenjl commented on a change in pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713#discussion_r458315193



##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.AuthResource;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+
+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 URLFactory. This constructor allows unit tests to mock server communication. */
+  /* package private */  ThirdEyeRcaRestClient(HttpURLConnectionFactory connectionFactory, ThirdEyePrincipal principal) {
+    super(connectionFactory);
+    this.principal = principal;
+    this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+  }
+
+  public Map<String, Object> getAllHighlights(long anomalyId) throws IOException {

Review comment:
       Add comments to the public methods of this class.

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.AuthResource;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+
+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 URLFactory. This constructor allows unit tests to mock server communication. */
+  /* package private */  ThirdEyeRcaRestClient(HttpURLConnectionFactory connectionFactory, ThirdEyePrincipal principal) {
+    super(connectionFactory);
+    this.principal = principal;
+    this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+  }
+
+  public Map<String, Object> getAllHighlights(long anomalyId) throws IOException {
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put("anomalyId", String.valueOf(anomalyId));
+
+    Map<String, String> headers = new HashMap<>();
+    headers.put("Cookie", AuthResource.AUTH_TOKEN_NAME + "=" + principal.getSessionKey());
+    return doGet(
+        composeUrl(this.thirdEyeHost, THIRDEYE_RCA_HIGHLIGHTS_URI, queryParameters),
+        this.thirdEyeHost,
+        headers,
+        new ResponseToMap());
+  }
+
+  public Map<String, Object> getCubeHighlights(long anomalyId) throws IOException {
+    MergedAnomalyResultDTO anomalyDTO = DAORegistry.getInstance().getMergedAnomalyResultDAO().findById(anomalyId);
+
+    long startTime = anomalyDTO.getStartTime();
+    long endTime = anomalyDTO.getEndTime();
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put(METRIC_URN, anomalyDTO.getMetricUrn());
+    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, "3");
+    queryParameters.put(CUBE_ONE_SIDE_ERROR, "false");
+    queryParameters.put(CUBE_ORDER_TYPE, "auto");
+    queryParameters.put(CUBE_SUMMARY_SIZE, "3");

Review comment:
       Should we make these parameters configurable instead of hardcoding them in the method? `ThirdEyeRcaRestClient` should be a generally-purposed REST client for RCA.

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java
##########
@@ -60,12 +65,51 @@
 
   private final List<RootCauseEntityFormatter> formatters;
   private final Map<String, RCAFramework> frameworks;
+  private final String dashboardHost;
 
-  public RootCauseResource(Map<String, RCAFramework> frameworks, List<RootCauseEntityFormatter> formatters) {
+  public RootCauseResource(String dashboardHost, Map<String, RCAFramework> frameworks, List<RootCauseEntityFormatter> formatters) {
+    this.dashboardHost = dashboardHost;
     this.frameworks = frameworks;
     this.formatters = formatters;
   }
 
+  @GET
+  @Path("/highlights")
+  @ApiOperation(value = "Send query")
+  public Map<String, String> highlights(
+      @ApiParam(value = "internal id of the anomaly")
+      @QueryParam("anomalyId") long anomalyId,
+      @Auth ThirdEyePrincipal principal) {
+    Map<String, String> responseMessage = new HashMap<>();
+    ThirdEyeRcaRestClient rcaClient = new ThirdEyeRcaRestClient(principal, dashboardHost);

Review comment:
       One thought on using REST client here. Although it makes sense to use REST calls to get information across different modules, for instance getting RCA info in notification module,  we should consider using method call directly rather than REST calls, since this endpoint is under RCA and all the following calls under this method are also going to RCA. Method calls can give us better performance and error handling. 

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {

Review comment:
       It would be nice if we rename or add comments to this class, saying that it is only used for short-lived HTTP connections. Developers should consider something else, if they need persistent HTTP connections for multiple calls.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] jihaozh commented on a change in pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

Posted by GitBox <gi...@apache.org>.
jihaozh commented on a change in pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713#discussion_r459140310



##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {

Review comment:
       Have we evaluated using the Jersey Client API library as the REST client? https://howtodoinjava.com/jersey/jersey-restful-client-examples/
   
   We are already using it Jersy to write the service. It seems natural to use it on the client-side as well. It can help us abstract away the low-level details of the client-server communication and avoid potential bugs. 

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {
+  private final Logger LOG = LoggerFactory.getLogger(this.getClass());
+
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private final HttpURLConnectionFactory connectionFactory;
+
+  /**
+   * Using a URLFactory to create URLs allows unit tests to mock out server communication.
+   */
+  public class HttpURLConnectionFactory {
+    public HttpURLConnection openConnection(String url) throws MalformedURLException, IOException {
+      return (HttpURLConnection) new URL(url).openConnection();
+    }
+  }
+
+  /**
+   * Set up the client with a default URLFactory that creates real HTTP connections.
+   * For unit tests, we use the alternate constructor to pass a mock.
+   */
+  public AbstractRestClient() {
+    connectionFactory = new HttpURLConnectionFactory();
+  }
+
+  /**
+   * For testing only, create a client with an alternate URLFactory. This constructor allows
+   * unit tests to mock server communication.
+   */
+  public AbstractRestClient(HttpURLConnectionFactory connectionFactory) {
+    this.connectionFactory = connectionFactory;
+  }
+
+  /**
+   * Perform a GET request to the given URL, accepts a method that will parse the response as a parameter.
+   * A timeout of zero is interpreted as an infinite timeout.
+   * @param <T>  the type parameter defined as the return type of the response parser method
+   * @param url the http url
+   * @param host the host to connect to
+   * @param headers the headers for communication
+   */
+  public <T> T doGet(String url, String host, Map<String, String> headers,
+      ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    return doGet(url, host, headers, 0, 0, responseParserFunc);
+  }
+
+  /**
+   * 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
+   * @param host the host to connect to
+   * @param connectTimeout timeout in milliseconds
+   * @param readTimeout timeout in milliseconds
+   */
+  public <T> T doGet(String url, String host, Map<String, String> reqHeaders, int connectTimeout, int readTimeout,
+      ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    Map<String, String> headers = new HashMap<>(reqHeaders);
+    headers.put("User-Agent", getClass().getName());
+    headers.put("Host", host);
+    headers.put("Accept", "application/json");
+
+    return doRequest(url, "GET", new byte[0], null, connectTimeout, readTimeout, headers,
+        responseParserFunc);
+  }
+
+  /**
+   * Perform a POST request to the given URL, with a JSON or raw string as content.
+   * If the payload is an object that will be serialized to JSON the isContentTypeJSON must be set to true
+   * @param <T>  the type parameter defined as the return type of the response parser method
+   * @param url the http url
+   * @param host the host to connect to
+   * @param postContent the post content
+   * @param isContentTypeJson flag indicating if the content is json type or not
+   */
+  public <T> T doPost(String url, String host, Map<String, String> reqHeaders, Object postContent,
+      boolean isContentTypeJson, ParseResponseFunction<InputStream, T> responseParserFunc) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    OBJECT_MAPPER.writeValue(baos, postContent);
+    byte[] content = baos.toByteArray();
+    String contentType = isContentTypeJson ? "application/json" : null;
+
+    Map<String, String> headers = new HashMap<>(reqHeaders);
+    headers.put("User-Agent", getClass().getName());
+    headers.put("Host", host);
+    headers.put("Accept", "application/json");
+
+    return doRequest(url, "POST", content, contentType, 0, 0, headers,
+        responseParserFunc);
+  }
+
+  /**
+   * Send a request to the given URL with the given parameters.
+   * @param <T> the type parameter defined as the return type of the response parser method
+   * @param url server url.
+   * @param method HTTP method.
+   * @param content request content.
+   * @param contentType request content type.
+   * @param connectTimeoutMillis connection timeout.
+   * @param readTimeoutMillis read timeout.
+   * @param headers any additional request headers.
+   * @return T the result of the parser functions applied to the response content.
+   */
+  private <T> T doRequest(String url, String method, byte[] content, String contentType, int connectTimeoutMillis,
+      int readTimeoutMillis, Map<String, String> headers, ParseResponseFunction<InputStream, T> responseParserFunc)
+      throws IOException {
+    long startedMillis = System.currentTimeMillis();
+
+    HttpURLConnection conn = this.connectionFactory.openConnection(url);

Review comment:
       shall the `connection` be put into a try-with-resources block? so that it can be closed whenever there is any exception.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] akshayrai commented on a change in pull request #5713: [TE] Setup a rest client pipeline for comms between ThirdEye services; added RCA highlights API as an example

Posted by GitBox <gi...@apache.org>.
akshayrai commented on a change in pull request #5713:
URL: https://github.com/apache/incubator-pinot/pull/5713#discussion_r459138989



##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/AbstractRestClient.java
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A generic API Rest Client to perform GET and POST
+ */
+public abstract class AbstractRestClient {

Review comment:
       Updated!

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.AuthResource;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+
+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 URLFactory. This constructor allows unit tests to mock server communication. */
+  /* package private */  ThirdEyeRcaRestClient(HttpURLConnectionFactory connectionFactory, ThirdEyePrincipal principal) {
+    super(connectionFactory);
+    this.principal = principal;
+    this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+  }
+
+  public Map<String, Object> getAllHighlights(long anomalyId) throws IOException {

Review comment:
       Added

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/common/restclient/ThirdEyeRcaRestClient.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.thirdeye.auth.ThirdEyePrincipal;
+import org.apache.pinot.thirdeye.dashboard.resources.v2.AuthResource;
+import org.apache.pinot.thirdeye.datalayer.dto.MergedAnomalyResultDTO;
+import org.apache.pinot.thirdeye.datasource.DAORegistry;
+
+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 URLFactory. This constructor allows unit tests to mock server communication. */
+  /* package private */  ThirdEyeRcaRestClient(HttpURLConnectionFactory connectionFactory, ThirdEyePrincipal principal) {
+    super(connectionFactory);
+    this.principal = principal;
+    this.thirdEyeHost = DEFAULT_THIRDEYE_RCA_SERVICE_HOST;
+  }
+
+  public Map<String, Object> getAllHighlights(long anomalyId) throws IOException {
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put("anomalyId", String.valueOf(anomalyId));
+
+    Map<String, String> headers = new HashMap<>();
+    headers.put("Cookie", AuthResource.AUTH_TOKEN_NAME + "=" + principal.getSessionKey());
+    return doGet(
+        composeUrl(this.thirdEyeHost, THIRDEYE_RCA_HIGHLIGHTS_URI, queryParameters),
+        this.thirdEyeHost,
+        headers,
+        new ResponseToMap());
+  }
+
+  public Map<String, Object> getCubeHighlights(long anomalyId) throws IOException {
+    MergedAnomalyResultDTO anomalyDTO = DAORegistry.getInstance().getMergedAnomalyResultDAO().findById(anomalyId);
+
+    long startTime = anomalyDTO.getStartTime();
+    long endTime = anomalyDTO.getEndTime();
+    TreeMap<String, String> queryParameters = new TreeMap<String, String>();
+    queryParameters.put(METRIC_URN, anomalyDTO.getMetricUrn());
+    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, "3");
+    queryParameters.put(CUBE_ONE_SIDE_ERROR, "false");
+    queryParameters.put(CUBE_ORDER_TYPE, "auto");
+    queryParameters.put(CUBE_SUMMARY_SIZE, "3");

Review comment:
       Passed them as parameters. Thanks

##########
File path: thirdeye/thirdeye-pinot/src/main/java/org/apache/pinot/thirdeye/dashboard/resources/v2/RootCauseResource.java
##########
@@ -60,12 +65,51 @@
 
   private final List<RootCauseEntityFormatter> formatters;
   private final Map<String, RCAFramework> frameworks;
+  private final String dashboardHost;
 
-  public RootCauseResource(Map<String, RCAFramework> frameworks, List<RootCauseEntityFormatter> formatters) {
+  public RootCauseResource(String dashboardHost, Map<String, RCAFramework> frameworks, List<RootCauseEntityFormatter> formatters) {
+    this.dashboardHost = dashboardHost;
     this.frameworks = frameworks;
     this.formatters = formatters;
   }
 
+  @GET
+  @Path("/highlights")
+  @ApiOperation(value = "Send query")
+  public Map<String, String> highlights(
+      @ApiParam(value = "internal id of the anomaly")
+      @QueryParam("anomalyId") long anomalyId,
+      @Auth ThirdEyePrincipal principal) {
+    Map<String, String> responseMessage = new HashMap<>();
+    ThirdEyeRcaRestClient rcaClient = new ThirdEyeRcaRestClient(principal, dashboardHost);

Review comment:
       good point! I have updated accordingly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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