You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/10/05 21:41:29 UTC

[10/23] hive git commit: HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, reviewed by Sergio Pena)

HIVE-11913 : Verify existence of tests for new changes in HiveQA (Szehon, reviewed by Sergio Pena)


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

Branch: refs/heads/llap
Commit: bbb312f36b81b95ef6b0b003606799aaffe30142
Parents: bc1c434
Author: Szehon Ho <sz...@cloudera.com>
Authored: Fri Oct 2 14:20:31 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Fri Oct 2 14:21:14 2015 -0700

----------------------------------------------------------------------
 .../hive/ptest/execution/JIRAService.java       |  115 +-
 .../org/apache/hive/ptest/execution/PTest.java  |   11 +-
 .../hive/ptest/execution/TestCheckPhase.java    |   77 +
 .../ptest/execution/TestTestCheckPhase.java     |   91 +
 .../src/test/resources/HIVE-10761.6.patch       | 2539 ++++++++++++++++++
 .../src/test/resources/HIVE-11271.4.patch       |  606 +++++
 .../ptest2/src/test/resources/HIVE-9377.1.patch |   25 +
 .../ptest2/src/test/resources/remove-test.patch |   33 +
 8 files changed, 3447 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
index c7be572..37127ea 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/JIRAService.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.*;
 
+import com.google.common.collect.Sets;
 import org.apache.commons.cli.*;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.hive.ptest.api.server.TestLogger;
@@ -94,7 +95,12 @@ class JIRAService {
   }
 
   void postComment(boolean error, int numTestsExecuted, SortedSet<String> failedTests,
-      List<String> messages) {
+    List<String> messages) {
+    postComment(error, numTestsExecuted, failedTests, messages, new HashSet<String>());
+  }
+
+  void postComment(boolean error, int numTestsExecuted, SortedSet<String> failedTests,
+    List<String> messages, Set<String> addedTests) {
     DefaultHttpClient httpClient = new DefaultHttpClient();
     try {
       BuildInfo buildInfo = formatBuildTag(mBuildTag);
@@ -102,9 +108,9 @@ class JIRAService {
       List<String> comments = Lists.newArrayList();
       comments.add("");
       comments.add("");
-      if(!failedTests.isEmpty()) {
+      if (!failedTests.isEmpty()) {
         comments.add("{color:red}Overall{color}: -1 at least one tests failed");
-      } else if(numTestsExecuted == 0) {
+      } else if (numTestsExecuted == 0) {
         comments.add("{color:red}Overall{color}: -1 no tests executed");
       } else if (error) {
         comments.add("{color:red}Overall{color}: -1 build exited with an error");
@@ -112,17 +118,23 @@ class JIRAService {
         comments.add("{color:green}Overall{color}: +1 all checks pass");
       }
       comments.add("");
-      if(!mPatch.isEmpty()) {
+      if (!mPatch.isEmpty()) {
         comments.add("Here are the results of testing the latest attachment:");
         comments.add(mPatch);
       }
       comments.add("");
-      if(numTestsExecuted > 0) {
+      if (addedTests.size() > 0) {
+        comments.add(formatSuccess("+1 due to " + addedTests.size() + " test(s) being added or modified."));
+      } else {
+        comments.add(formatError("-1 due to no test(s) being added or modified."));
+      }
+      comments.add("");
+      if (numTestsExecuted > 0) {
         if (failedTests.isEmpty()) {
-          comments.add(formatSuccess("+1 "+ numTestsExecuted + " tests passed"));
+          comments.add(formatSuccess("+1 " + numTestsExecuted + " tests passed"));
         } else {
           comments.add(formatError("-1 due to " + failedTests.size()
-              + " failed/errored test(s), " + numTestsExecuted + " tests executed"));
+            + " failed/errored test(s), " + numTestsExecuted + " tests executed"));
           comments.add("*Failed tests:*");
           comments.add("{noformat}");
           comments.addAll(failedTests);
@@ -131,12 +143,12 @@ class JIRAService {
         comments.add("");
       }
       comments.add("Test results: " + mJenkinsURL + "/" +
-          buildInfo.getFormattedBuildTag() + "/testReport");
+        buildInfo.getFormattedBuildTag() + "/testReport");
       comments.add("Console output: " + mJenkinsURL + "/" +
-          buildInfo.getFormattedBuildTag() + "/console");
+        buildInfo.getFormattedBuildTag() + "/console");
       comments.add("Test logs: " + mLogsURL + buildTagForLogs);
       comments.add("");
-      if(!messages.isEmpty()) {
+      if (!messages.isEmpty()) {
         comments.add("Messages:");
         comments.add("{noformat}");
         comments.addAll(trimMessages(messages));
@@ -147,16 +159,16 @@ class JIRAService {
       String attachmentId = parseAttachementId(mPatch);
       comments.add("");
       comments.add("ATTACHMENT ID: " + attachmentId +
-          " - " + buildInfo.getBuildName());
+        " - " + buildInfo.getBuildName());
       mLogger.info("Comment: " + Joiner.on("\n").join(comments));
       String body = Joiner.on("\n").join(comments);
       String url = String.format("%s/rest/api/2/issue/%s/comment", mUrl, mName);
       URL apiURL = new URL(mUrl);
       httpClient.getCredentialsProvider()
-      .setCredentials(
+        .setCredentials(
           new AuthScope(apiURL.getHost(), apiURL.getPort(),
-              AuthScope.ANY_REALM),
-              new UsernamePasswordCredentials(mUser, mPassword));
+            AuthScope.ANY_REALM),
+          new UsernamePasswordCredentials(mUser, mPassword));
       BasicHttpContext localcontext = new BasicHttpContext();
       localcontext.setAttribute("preemptive-auth", new BasicScheme());
       httpClient.addRequestInterceptor(new PreemptiveAuth(), 0);
@@ -169,36 +181,42 @@ class JIRAService {
       StatusLine statusLine = httpResponse.getStatusLine();
       if (statusLine.getStatusCode() != 201) {
         throw new RuntimeException(statusLine.getStatusCode() + " "
-            + statusLine.getReasonPhrase());
+          + statusLine.getReasonPhrase());
       }
       mLogger.info("JIRA Response Metadata: " + httpResponse);
     } catch (Exception e) {
       mLogger.error("Encountered error attempting to post comment to " + mName,
-          e);
+        e);
     } finally {
       httpClient.getConnectionManager().shutdown();
     }
   }
+
   static List<String> trimMessages(List<String> messages) {
     int size = messages.size();
-    if(size > MAX_MESSAGES) {
+    if (size > MAX_MESSAGES) {
       messages = messages.subList(size - MAX_MESSAGES, size);
       messages.add(0, TRIMMED_MESSAGE);
     }
     return messages;
   }
+
   @SuppressWarnings("unused")
   private static class Body {
     private String body;
+
     public Body() {
 
     }
+
     public Body(String body) {
       this.body = body;
     }
+
     public String getBody() {
       return body;
     }
+
     public void setBody(String body) {
       this.body = body;
     }
@@ -209,7 +227,7 @@ class JIRAService {
     private String buildName;
     private String formattedBuildTag;
 
-    public BuildInfo (String buildName, String formattedBuildTag) {
+    public BuildInfo(String buildName, String formattedBuildTag) {
       this.buildName = buildName;
       this.formattedBuildTag = formattedBuildTag;
     }
@@ -228,7 +246,7 @@ class JIRAService {
    */
   @VisibleForTesting
   static BuildInfo formatBuildTag(String buildTag) {
-    if(buildTag.contains("-")) {
+    if (buildTag.contains("-")) {
       int lastDashIndex = buildTag.lastIndexOf("-");
       String buildName = buildTag.substring(0, lastDashIndex);
       String buildId = buildTag.substring(lastDashIndex + 1);
@@ -237,6 +255,7 @@ class JIRAService {
     }
     throw new IllegalArgumentException("Build tag '" + buildTag + "' must contain a -");
   }
+
   static String formatBuildTagForLogs(String buildTag) {
     if (buildTag.endsWith("/")) {
       return buildTag;
@@ -244,6 +263,7 @@ class JIRAService {
       return buildTag + "/";
     }
   }
+
   private static String formatError(String msg) {
     return String.format("{color:red}ERROR:{color} %s", msg);
   }
@@ -255,7 +275,7 @@ class JIRAService {
   static class PreemptiveAuth implements HttpRequestInterceptor {
 
     public void process(final HttpRequest request, final HttpContext context)
-        throws HttpException, IOException {
+      throws HttpException, IOException {
       AuthState authState = (AuthState) context.getAttribute(ClientContext.TARGET_AUTH_STATE);
       if (authState.getAuthScheme() == null) {
         AuthScheme authScheme = (AuthScheme) context.getAttribute("preemptive-auth");
@@ -263,34 +283,35 @@ class JIRAService {
         HttpHost targetHost = (HttpHost) context.getAttribute(ExecutionContext.HTTP_TARGET_HOST);
         if (authScheme != null) {
           Credentials creds = credsProvider.getCredentials(new AuthScope(
-              targetHost.getHostName(), targetHost.getPort()));
+            targetHost.getHostName(), targetHost.getPort()));
           if (creds == null) {
             throw new HttpException(
-                "No credentials for preemptive authentication");
+              "No credentials for preemptive authentication");
           }
           authState.update(authScheme, creds);
         }
       }
     }
   }
+
   private static String parseAttachementId(String patch) {
-    if(patch == null) {
+    if (patch == null) {
       return "";
     }
     String result = FilenameUtils.getPathNoEndSeparator(patch.trim());
-    if(result == null) {
+    if (result == null) {
       return "";
     }
     result = FilenameUtils.getName(result.trim());
-    if(result == null) {
+    if (result == null) {
       return "";
     }
     return result.trim();
   }
 
   private static void assertRequired(CommandLine commandLine, String[] requiredOptions) throws IllegalArgumentException {
-    for(String requiredOption : requiredOptions) {
-      if(!commandLine.hasOption(requiredOption)) {
+    for (String requiredOption : requiredOptions) {
+      if (!commandLine.hasOption(requiredOption)) {
         throw new IllegalArgumentException("--" + requiredOption + " is required");
       }
     }
@@ -311,7 +332,7 @@ class JIRAService {
   private static final String FIELD_FAILED_TESTS = "failedTests";
   private static final String FIELD_MESSAGES = "messages";
   private static final String FIELD_JIRA_USER = "jiraUser";
-  private static final String FIELD_JIRA_PASS= "jiraPassword";
+  private static final String FIELD_JIRA_PASS = "jiraPassword";
 
   private static Map<String, Class> supportedJsonFields = new HashMap<String, Class>() {
     {
@@ -387,9 +408,9 @@ class JIRAService {
     }
 
     assertRequired(cmd, new String[]{
-        OPT_USER_LONG,
-        OPT_PASS_LONG,
-        OPT_FILE_LONG
+      OPT_USER_LONG,
+      OPT_PASS_LONG,
+      OPT_FILE_LONG
     });
 
     return cmd;
@@ -400,7 +421,7 @@ class JIRAService {
 
     try {
       cmd = parseCommandLine(args);
-    } catch(ParseException e) {
+    } catch (ParseException e) {
       System.out.println("Error parsing command arguments: " + e.getMessage());
       System.exit(1);
     }
@@ -413,25 +434,25 @@ class JIRAService {
     Map<String, Object> jsonValues = parseJsonFile(cmd.getOptionValue(OPT_FILE_LONG));
 
     Map<String, String> context = Maps.newHashMap();
-    context.put(FIELD_JIRA_URL, (String)jsonValues.get(FIELD_JIRA_URL));
+    context.put(FIELD_JIRA_URL, (String) jsonValues.get(FIELD_JIRA_URL));
     context.put(FIELD_JIRA_USER, cmd.getOptionValue(OPT_USER_LONG));
     context.put(FIELD_JIRA_PASS, cmd.getOptionValue(OPT_PASS_LONG));
-    context.put(FIELD_LOGS_URL, (String)jsonValues.get(FIELD_LOGS_URL));
-    context.put(FIELD_REPO, (String)jsonValues.get(FIELD_REPO));
-    context.put(FIELD_REPO_NAME, (String)jsonValues.get(FIELD_REPO_NAME));
-    context.put(FIELD_REPO_TYPE, (String)jsonValues.get(FIELD_REPO_TYPE));
-    context.put(FIELD_REPO_BRANCH, (String)jsonValues.get(FIELD_REPO_BRANCH));
-    context.put(FIELD_JENKINS_URL, (String)jsonValues.get(FIELD_JENKINS_URL));
+    context.put(FIELD_LOGS_URL, (String) jsonValues.get(FIELD_LOGS_URL));
+    context.put(FIELD_REPO, (String) jsonValues.get(FIELD_REPO));
+    context.put(FIELD_REPO_NAME, (String) jsonValues.get(FIELD_REPO_NAME));
+    context.put(FIELD_REPO_TYPE, (String) jsonValues.get(FIELD_REPO_TYPE));
+    context.put(FIELD_REPO_BRANCH, (String) jsonValues.get(FIELD_REPO_BRANCH));
+    context.put(FIELD_JENKINS_URL, (String) jsonValues.get(FIELD_JENKINS_URL));
 
     TestLogger logger = new TestLogger(System.err, TestLogger.LEVEL.TRACE);
     TestConfiguration configuration = new TestConfiguration(new Context(context), logger);
-    configuration.setJiraName((String)jsonValues.get(FIELD_JIRA_NAME));
-    configuration.setPatch((String)jsonValues.get(FIELD_PATCH_URL));
-
-    JIRAService service = new JIRAService(logger, configuration, (String)jsonValues.get(FIELD_BUILD_TAG));
-    List<String> messages = (List)jsonValues.get(FIELD_MESSAGES);
-    SortedSet<String> failedTests = (SortedSet)jsonValues.get(FIELD_FAILED_TESTS);
-    boolean error = (Integer)jsonValues.get(FIELD_BUILD_STATUS) == 0 ? false : true;
-    service.postComment(error, (Integer)jsonValues.get(FIELD_NUM_TESTS_EXECUTED), failedTests, messages);
+    configuration.setJiraName((String) jsonValues.get(FIELD_JIRA_NAME));
+    configuration.setPatch((String) jsonValues.get(FIELD_PATCH_URL));
+
+    JIRAService service = new JIRAService(logger, configuration, (String) jsonValues.get(FIELD_BUILD_TAG));
+    List<String> messages = (List) jsonValues.get(FIELD_MESSAGES);
+    SortedSet<String> failedTests = (SortedSet) jsonValues.get(FIELD_FAILED_TESTS);
+    boolean error = (Integer) jsonValues.get(FIELD_BUILD_STATUS) == 0 ? false : true;
+    service.postComment(error, (Integer) jsonValues.get(FIELD_NUM_TESTS_EXECUTED), failedTests, messages);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
index 7217ef9..35cc752 100644
--- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/PTest.java
@@ -23,6 +23,7 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -72,6 +73,7 @@ public class PTest {
 
   private final TestConfiguration mConfiguration;
   private final ListeningExecutorService mExecutor;
+  private final Set<String> mAddedTests;
   private final Set<String> mExecutedTests;
   private final Set<String> mFailedTests;
   private final List<Phase> mPhases;
@@ -92,6 +94,7 @@ public class PTest {
     mBuildTag = buildTag;
     mExecutedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
     mFailedTests = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+    mAddedTests = new HashSet<String>();
     mExecutionContext = executionContext;
     mSshCommandExecutor = sshCommandExecutor;
     mRsyncCommandExecutor = rsyncCommandExecutor;
@@ -148,6 +151,7 @@ public class PTest {
     }
     mHostExecutors = new CopyOnWriteArrayList<HostExecutor>(hostExecutors);
     mPhases = Lists.newArrayList();
+    mPhases.add(new TestCheckPhase(mHostExecutors, localCommandFactory, templateDefaults, patchFile, logger, mAddedTests));
     mPhases.add(new PrepPhase(mHostExecutors, localCommandFactory, templateDefaults, scratchDir, patchFile, logger));
     mPhases.add(new ExecutionPhase(mHostExecutors, mExecutionContext, hostExecutorBuilder, localCommandFactory, templateDefaults,
         succeededLogDir, failedLogDir, testParser.parse(), mExecutedTests, mFailedTests, logger));
@@ -213,7 +217,7 @@ public class PTest {
       for(Map.Entry<String, Long> entry : elapsedTimes.entrySet()) {
         mLogger.info(String.format("PERF: Phase %s took %d minutes", entry.getKey(), entry.getValue()));
       }
-      publishJiraComment(error, messages, failedTests);
+      publishJiraComment(error, messages, failedTests, mAddedTests);
       if(error || !mFailedTests.isEmpty()) {
         result = 1;
       }
@@ -221,7 +225,7 @@ public class PTest {
     return result;
   }
 
-  private void publishJiraComment(boolean error, List<String> messages, SortedSet<String> failedTests) {
+  private void publishJiraComment(boolean error, List<String> messages, SortedSet<String> failedTests, Set<String> addedTests) {
     if(mConfiguration.getJiraName().isEmpty()) {
       mLogger.info("Skipping JIRA comment as name is empty.");
       return;
@@ -238,8 +242,9 @@ public class PTest {
       mLogger.info("Skipping JIRA comment as password is empty.");
       return;
     }
+    mLogger.info("Added tests: " + addedTests);
     JIRAService jira = new JIRAService(mLogger, mConfiguration, mBuildTag);
-    jira.postComment(error, mExecutedTests.size(), failedTests, messages);
+    jira.postComment(error, mExecutedTests.size(), failedTests, messages, addedTests);
   }
 
   public static class Builder {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
new file mode 100644
index 0000000..1107dcd
--- /dev/null
+++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/TestCheckPhase.java
@@ -0,0 +1,77 @@
+/*
+ * 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.execution;
+
+import com.google.common.collect.ImmutableMap;
+import org.slf4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class TestCheckPhase extends Phase {
+  private final File mPatchFile;
+  private Set<String> modifiedTestFiles;
+
+  private static final Pattern fileNameFromDiff = Pattern.compile("[/][^\\s]*");
+  private static final Pattern javaTest = Pattern.compile("Test.*java");
+
+  public TestCheckPhase(List<HostExecutor> hostExecutors,
+    LocalCommandFactory localCommandFactory,
+    ImmutableMap<String, String> templateDefaults,
+    File patchFile, Logger logger, Set<String> modifiedTestFiles) {
+    super(hostExecutors, localCommandFactory, templateDefaults, logger);
+    this.mPatchFile = patchFile;
+    this.modifiedTestFiles = modifiedTestFiles;
+  }
+  @Override
+  public void execute() throws Exception {
+    if(mPatchFile != null) {
+      logger.info("Reading patchfile " + mPatchFile.getAbsolutePath());
+      FileReader fr = null;
+      try {
+        fr = new FileReader(mPatchFile);
+        BufferedReader br = new BufferedReader(fr);
+        String line;
+        while ((line = br.readLine()) != null) {
+          if(line.startsWith("+++")) {
+            logger.info("Searching line : " + line);
+            Matcher fileNameMatcher = fileNameFromDiff.matcher(line);
+            if (fileNameMatcher.find()) {
+              String filePath = fileNameMatcher.group(0);
+              String fileName = filePath.substring(filePath.lastIndexOf("/")+1);
+              Matcher javaTestMatcher = javaTest.matcher(fileName);
+              if (javaTestMatcher.find() || fileName.endsWith(".q")) {
+                modifiedTestFiles.add(fileName);
+              }
+            }
+          }
+        }
+      } finally {
+        fr.close();
+      }
+    } else {
+      logger.error("Patch file is null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bbb312f3/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
new file mode 100644
index 0000000..7183ee3
--- /dev/null
+++ b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestTestCheckPhase.java
@@ -0,0 +1,91 @@
+/*
+ * 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.execution;
+
+import org.approvaltests.Approvals;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.URL;
+import java.util.HashSet;
+import java.util.Set;
+
+public class TestTestCheckPhase extends AbstractTestPhase {
+  private TestCheckPhase phase;
+
+  @Before
+  public void setup() throws Exception {
+    initialize(getClass().getSimpleName());
+    createHostExecutor();
+  }
+  @Test
+  public void testNoTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-9377.1.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 0);
+  }
+
+
+  @Test
+  public void testJavaTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-10761.6.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 3);
+    Assert.assertTrue(addedTests.contains("TestCodahaleMetrics.java"));
+    Assert.assertTrue(addedTests.contains("TestMetaStoreMetrics.java"));
+    Assert.assertTrue(addedTests.contains("TestLegacyMetrics.java"));
+  }
+
+  @Test
+  public void testQTests() throws Exception {
+    URL url = this.getClass().getResource("/HIVE-11271.4.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 1);
+    Assert.assertTrue(addedTests.contains("unionall_unbalancedppd.q"));
+  }
+
+  @Test
+  public void testRemoveTest() throws Exception {
+    URL url = this.getClass().getResource("/remove-test.patch");
+    File patchFile = new File(url.getFile());
+    Set<String> addedTests = new HashSet<String>();
+    phase = new TestCheckPhase(hostExecutors, localCommandFactory,
+      templateDefaults, patchFile, logger, addedTests);
+    phase.execute();
+
+    Assert.assertEquals(addedTests.size(), 0);
+  }
+}