You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pv...@apache.org on 2018/04/11 07:46:34 UTC

hive git commit: HIVE-19077: Handle duplicate ptests requests standing in queue at the same time (Adam Szita, via Peter Vary)

Repository: hive
Updated Branches:
  refs/heads/master f6dbcac35 -> 7c1f01004


HIVE-19077: Handle duplicate ptests requests standing in queue at the same time (Adam Szita, via Peter Vary)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/7c1f0100
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7c1f0100
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7c1f0100

Branch: refs/heads/master
Commit: 7c1f010045db3af6fa5b805ae4f49fc9b07d5639
Parents: f6dbcac
Author: Adam Szita <sz...@cloudera.com>
Authored: Wed Apr 11 09:45:56 2018 +0200
Committer: Peter Vary <pv...@cloudera.com>
Committed: Wed Apr 11 09:45:56 2018 +0200

----------------------------------------------------------------------
 dev-support/jenkins-common.sh                   |   2 +
 dev-support/jenkins-execute-build.sh            |   3 +-
 .../hive/ptest/api/client/JenkinsQueueUtil.java | 133 +++++++++++++++++++
 .../hive/ptest/api/client/PTestClient.java      |  15 ++-
 4 files changed, 150 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7c1f0100/dev-support/jenkins-common.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-common.sh b/dev-support/jenkins-common.sh
index 0467d11..64f486f 100644
--- a/dev-support/jenkins-common.sh
+++ b/dev-support/jenkins-common.sh
@@ -15,6 +15,8 @@
 # limitations under the License.
 
 JIRA_ROOT_URL="https://issues.apache.org"
+JENKINS_URL="https://builds.apache.org"
+JENKINS_QUEUE_QUERY="/queue/api/json?tree=items[task[name],inQueueSince,actions[parameters[name,value]],why]"
 
 fail() {
   echo "$@" 1>&2

http://git-wip-us.apache.org/repos/asf/hive/blob/7c1f0100/dev-support/jenkins-execute-build.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-execute-build.sh b/dev-support/jenkins-execute-build.sh
index f660fcb..35392dd 100644
--- a/dev-support/jenkins-execute-build.sh
+++ b/dev-support/jenkins-execute-build.sh
@@ -51,7 +51,8 @@ call_ptest_server() {
 	local PTEST_CLASSPATH="$PTEST_BUILD_DIR/hive/testutils/ptest2/target/hive-ptest-3.0-classes.jar:$PTEST_BUILD_DIR/hive/testutils/ptest2/target/lib/*"
 
 	java -cp "$PTEST_CLASSPATH" org.apache.hive.ptest.api.client.PTestClient --command testStart \
-		--outputDir "$PTEST_BUILD_DIR/hive/testutils/ptest2/target" --password "$JIRA_PASSWORD" "$@"
+		--outputDir "$PTEST_BUILD_DIR/hive/testutils/ptest2/target" --password "$JIRA_PASSWORD" \
+		--jenkinsQueueUrl "$JENKINS_URL$JENKINS_QUEUE_QUERY" "$@"
 }
 
 # Unpack all test results

http://git-wip-us.apache.org/repos/asf/hive/blob/7c1f0100/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/JenkinsQueueUtil.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/JenkinsQueueUtil.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/JenkinsQueueUtil.java
new file mode 100644
index 0000000..f973781
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/JenkinsQueueUtil.java
@@ -0,0 +1,133 @@
+/*
+ * 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.hive.ptest.api.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.http.HttpResponse;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.util.EntityUtils;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+
+/**
+ * Utility class for the Precommit test job queue on Jenkins
+ */
+public class JenkinsQueueUtil {
+
+  private static final String JSON_ITEMS_FIELD = "items";
+  private static final String JSON_TASK_FIELD = "task";
+  private static final String JSON_TASK_NAME_FIELD = "name";
+  private static final String JSON_PARAMETERS_FIELD = "parameters";
+  private static final String JSON_PARAMETER_NAME_FIELD = "name";
+  private static final String JSON_PARAMETER_VALUE_FIELD = "value";
+
+  private static final String JOB_NAME = "PreCommit-HIVE-Build";
+  private static final String ISSUE_FIELD_KEY = "ISSUE_NUM";
+  private static final String JIRA_KEY_PREFIX = "HIVE-";
+
+
+  /**
+   * Looks up the current queue of the precommit job on a jenkins instance (specified by
+   * PTestClient.JENKINS_QUEUE_URL), and checks if current Jira is standing in queue already (i.e.
+   * will be executed in the future too)
+   *
+   * @param commandLine PTestClient's command line option values' list
+   * @return whether or not the Jira specified in the command line can be found in the job queue
+   */
+  public static boolean isJiraAlreadyInQueue(CommandLine commandLine) {
+    if (!(commandLine.hasOption(PTestClient.JENKINS_QUEUE_URL) &&
+            commandLine.hasOption(PTestClient.JIRA))) {
+      return false;
+    }
+    try {
+      System.out.println("Checking " + JOB_NAME + " queue...");
+      String queueJson = httpGet(commandLine.getOptionValue(PTestClient.JENKINS_QUEUE_URL));
+      List<String> jirasInQueue = parseJiras(queueJson);
+      if (jirasInQueue.size() > 0) {
+        System.out.println(JOB_NAME + " has the following jira(s) in queue: " + jirasInQueue);
+      } else {
+        return false;
+      }
+
+      String jira = commandLine.getOptionValue(PTestClient.JIRA).replaceAll(JIRA_KEY_PREFIX,"");
+      if (jirasInQueue.contains(jira)) {
+        return true;
+      }
+
+    } catch (IOException e) {
+      System.err.println("Error checking " + JOB_NAME + " build queue: " + e);
+    }
+    return false;
+  }
+
+  /**
+   * Parses raw json to produce a list of Jira number strings.
+   * @param queueJson
+   * @return
+   * @throws IOException
+   */
+  private static List<String> parseJiras(String queueJson) throws IOException {
+    List<String> jirasInQueue = new ArrayList<>();
+    ObjectMapper objectMapper = new ObjectMapper();
+    JsonNode rootNode = objectMapper.readTree(queueJson);
+    List<JsonNode> items = Lists.newArrayList(rootNode.findValue(JSON_ITEMS_FIELD).iterator());
+    for (JsonNode item : items) {
+      String taskName = item.path(JSON_TASK_FIELD).path(JSON_TASK_NAME_FIELD).asText();
+      if (JOB_NAME.equals(taskName)) {
+        List<JsonNode> parameters = Lists.newArrayList(item.findValue(JSON_PARAMETERS_FIELD));
+        for (JsonNode parameter : parameters) {
+          if (ISSUE_FIELD_KEY.equals(parameter.path(JSON_PARAMETER_NAME_FIELD).asText())) {
+            jirasInQueue.add(parameter.path(JSON_PARAMETER_VALUE_FIELD).asText());
+          }
+        }
+      }
+    }
+    return jirasInQueue;
+  }
+
+  private static String httpGet(String url)
+          throws IOException {
+    DefaultHttpClient httpClient = new DefaultHttpClient();
+    HttpGet request = new HttpGet(url);
+    try {
+      request.addHeader("content-type", "application/json");
+      httpClient.setHttpRequestRetryHandler(new PTestClient.PTestHttpRequestRetryHandler());
+      HttpResponse httpResponse = httpClient.execute(request);
+      StatusLine statusLine = httpResponse.getStatusLine();
+      if (statusLine.getStatusCode() != 200) {
+        throw new IllegalStateException(statusLine.getStatusCode() + " " + statusLine.getReasonPhrase());
+      }
+      String response = EntityUtils.toString(httpResponse.getEntity(), "UTF-8");
+      return response;
+    } finally {
+      request.abort();
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7c1f0100/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
index b72a971..9970c36 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/api/client/PTestClient.java
@@ -81,10 +81,11 @@ public class PTestClient {
   private static final String PASSWORD = "password";
   private static final String PROFILE = "profile";
   private static final String PATCH = "patch";
-  private static final String JIRA = "jira";
+  public static final String JIRA = "jira";
   private static final String OUTPUT_DIR = "outputDir";
   private static final String TEST_HANDLE = "testHandle";
   private static final String CLEAR_LIBRARY_CACHE = "clearLibraryCache";
+  public static final String JENKINS_QUEUE_URL = "jenkinsQueueUrl";
   private static final int MAX_RETRIES = 10;
   private final String mApiEndPoint;
   private final String mLogsEndpoint;
@@ -242,7 +243,7 @@ public class PTestClient {
       request.abort();
     }
   }
-  private static class PTestHttpRequestRetryHandler implements HttpRequestRetryHandler {
+  public static class PTestHttpRequestRetryHandler implements HttpRequestRetryHandler {
     @Override
     public boolean retryRequest(IOException exception, int executionCount,
         HttpContext context) {
@@ -298,6 +299,7 @@ public class PTestClient {
     options.addOption(null, OUTPUT_DIR, true, "Directory to download and save test-results.tar.gz to. (Optional for testStart)");
     options.addOption(null, CLEAR_LIBRARY_CACHE, false, "Before starting the test, delete the ivy and maven directories (Optional for testStart)");
     options.addOption(null, LOGS_ENDPOINT, true, "URL to get the logs");
+    options.addOption(null, JENKINS_QUEUE_URL, true, "URL for quering Jenkins job queue");
 
     CommandLine commandLine = parser.parse(options, args);
 
@@ -319,6 +321,14 @@ public class PTestClient {
           PROFILE,
           TEST_HANDLE
       });
+
+      boolean jiraAlreadyInQueue = JenkinsQueueUtil.isJiraAlreadyInQueue(commandLine);
+      if (jiraAlreadyInQueue) {
+        System.out.println("Skipping ptest execution, as " + commandLine.getOptionValue(JIRA) +
+                " is scheduled in " + "queue in " + "the future too.");
+        System.exit(0);
+      }
+
       result = client.testStart(commandLine.getOptionValue(PROFILE), commandLine.getOptionValue(TEST_HANDLE),
           commandLine.getOptionValue(JIRA), commandLine.getOptionValue(PATCH),
           commandLine.hasOption(CLEAR_LIBRARY_CACHE));
@@ -335,4 +345,5 @@ public class PTestClient {
       System.exit(1);
     }
   }
+
 }