HIVE-14540. Add support in ptest to create batches for non qfile tests. 
(Siddharth Seth, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master
Commit: eb1262077062bdeede2067ff251b8811fb1efc29
Parents: 37abf79
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 16 08:52:40 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 16 08:52:40 2016 -0700

----------------------------------------------------------------------
 .../hive/ptest/execution/ExecutionPhase.java    |   8 +
 .../hive/ptest/execution/HostExecutor.java      |   7 +-
 .../org/apache/hive/ptest/execution/PTest.java  |   3 +-
 .../ptest/execution/conf/FileListProvider.java  |  31 +
 .../ptest/execution/conf/QFileTestBatch.java    |  20 +-
 .../hive/ptest/execution/conf/TestBatch.java    |  30 +-
 .../ptest/execution/conf/TestConfiguration.java |   4 +
 .../hive/ptest/execution/conf/TestParser.java   |  68 +-
 .../ptest/execution/conf/UnitTestBatch.java     |  91 ++-
 .../conf/UnitTestPropertiesParser.java          | 666 ++++++++++++++++++
 .../ptest2/src/main/resources/batch-exec.vm     |  10 +-
 .../ptest2/src/main/resources/source-prep.vm    |  20 +-
 .../execution/MockRSyncCommandExecutor.java     |   7 +
 .../ptest/execution/MockSSHCommandExecutor.java |   7 +
 .../ptest/execution/TestExecutionPhase.java     |  10 +-
 ...cutionPhase.testFailingUnitTest.approved.txt |   6 +-
 ...cutionPhase.testPassingUnitTest.approved.txt |   6 +-
 .../hive/ptest/execution/TestHostExecutor.java  |  52 +-
 .../TestHostExecutor.testBasic.approved.txt     |  24 +-
 ...xecutor.testIsolatedFailsOnExec.approved.txt |  10 +-
 ...tor.testIsolatedFailsOnRsyncOne.approved.txt |   8 +-
 ...testIsolatedFailsOnRsyncUnknown.approved.txt |   8 +-
 ...xecutor.testParallelFailsOnExec.approved.txt |  10 +-
 ...ecutor.testParallelFailsOnRsync.approved.txt |   8 +-
 ...tScripts.testAlternativeTestJVM.approved.txt |  10 +-
 .../TestScripts.testBatch.approved.txt          |  10 +-
 .../TestScripts.testPrepGit.approved.txt        |  13 +-
 .../TestScripts.testPrepHadoop1.approved.txt    |  20 +-
 .../TestScripts.testPrepNone.approved.txt       |  13 +-
 .../TestScripts.testPrepSvn.approved.txt        |  13 +-
 .../execution/conf/TestQFileTestBatch.java      |   9 +-
 .../ptest/execution/conf/TestTestParser.java    |  10 +-
 .../conf/TestUnitTestPropertiesParser.java      | 672 +++++++++++++++++++
 .../ptest2/src/test/resources/log4j2.properties |  62 ++
 .../resources/test-configuration2.properties    | 154 +++++
 35 files changed, 1919 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
index 6063afc..65af6fa 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/ExecutionPhase.java
@@ -88,6 +88,14 @@ public class ExecutionPhase extends Phase {
     }
     logger.info("ParallelWorkQueueSize={}, IsolatedWorkQueueSize={}", 
parallelWorkQueue.size(),
         isolatedWorkQueue.size());
+    if (logger.isDebugEnabled()) {
+      for (TestBatch testBatch : parallelWorkQueue) {
+        logger.debug("PBatch: {}", testBatch);
+      }
+      for (TestBatch testBatch : isolatedWorkQueue) {
+        logger.debug("IBatch: {}", testBatch);
+      }
+    }
     try {
       int expectedNumHosts = hostExecutors.size();
       initalizeHosts();

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
index 5f84f00..123e310 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/HostExecutor.java
@@ -30,6 +30,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Stopwatch;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hive.ptest.execution.conf.Host;
 import org.apache.hive.ptest.execution.conf.TestBatch;
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
@@ -40,7 +41,6 @@ import org.apache.hive.ptest.execution.ssh.SSHCommand;
 import org.apache.hive.ptest.execution.ssh.SSHCommandExecutor;
 import org.apache.hive.ptest.execution.ssh.SSHExecutionException;
 import org.apache.hive.ptest.execution.ssh.SSHResult;
-import org.apache.logging.log4j.util.Strings;
 import org.slf4j.Logger;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -244,9 +244,8 @@ class HostExecutor {
     templateVariables.put("testArguments", batch.getTestArguments());
     templateVariables.put("localDir", drone.getLocalDirectory());
     templateVariables.put("logDir", drone.getLocalLogDirectory());
-    if (!Strings.isEmpty(batch.getTestModule())) {
-      templateVariables.put("testModule", batch.getTestModule());
-    }
+    
Preconditions.checkArgument(StringUtils.isNotBlank(batch.getTestModuleRelativeDir()));
+    templateVariables.put("testModule", batch.getTestModuleRelativeDir());
     String command = Templates.getTemplateResult("bash 
$localDir/$instanceName/scratch/" + script.getName(),
         templateVariables);
     Templates.writeTemplateResult("batch-exec.vm", script, templateVariables);

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/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 0ff090d..81bd4e3 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
@@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -135,7 +136,7 @@ public class PTest {
       templateDefaultsBuilder.put("additionalProfiles", 
configuration.getAdditionalProfiles());
     }
     templateDefaults = templateDefaultsBuilder.build();
-    TestParser testParser = new TestParser(configuration.getContext(), 
configuration.getTestCasePropertyName(),
+    TestParser testParser = new TestParser(configuration.getContext(), new 
AtomicInteger(1), configuration.getTestCasePropertyName(),
         new File(mExecutionContext.getLocalWorkingDirectory(), 
configuration.getRepositoryName() + "-source"),
         logger);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/FileListProvider.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/FileListProvider.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/FileListProvider.java
new file mode 100644
index 0000000..b1eb66f
--- /dev/null
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/FileListProvider.java
@@ -0,0 +1,31 @@
+/*
+ * 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.conf;
+
+import java.io.File;
+import java.util.Collection;
+
+// Exists primarily to allow for easier unit tests.
+public interface FileListProvider {
+
+  Collection<File> listFiles(
+      File directory, String[] extensions, boolean recursive);
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
index fa213db..fe4952c 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/QFileTestBatch.java
@@ -19,11 +19,12 @@
 package org.apache.hive.ptest.execution.conf;
 
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Iterators;
 
-public class QFileTestBatch implements TestBatch {
+public class QFileTestBatch extends TestBatch {
 
   private final String testCasePropertyName;
   private final String driver;
@@ -32,8 +33,11 @@ public class QFileTestBatch implements TestBatch {
   private final String moduleName;
   private final Set<String> tests;
   private final boolean isParallel;
-  public QFileTestBatch(String testCasePropertyName, String driver, 
-      String queryFilesProperty, Set<String> tests, boolean isParallel, String 
moduleName) {
+
+  public QFileTestBatch(AtomicInteger batchIdCounter, String 
testCasePropertyName, String driver,
+                        String queryFilesProperty, Set<String> tests, boolean 
isParallel,
+                        String moduleName) {
+    super(batchIdCounter);
     this.testCasePropertyName = testCasePropertyName;
     this.driver = driver;
     this.queryFilesProperty = queryFilesProperty;
@@ -66,7 +70,8 @@ public class QFileTestBatch implements TestBatch {
 
   @Override
   public String toString() {
-    return "QFileTestBatch [driver=" + driver + ", queryFilesProperty="
+    return "QFileTestBatch [batchId=" + getBatchId() + ", size=" + 
tests.size() + ", driver=" +
+        driver + ", queryFilesProperty="
         + queryFilesProperty + ", name=" + name + ", tests=" + tests
         + ", isParallel=" + isParallel + ", moduleName=" + moduleName + "]";
   }
@@ -76,11 +81,16 @@ public class QFileTestBatch implements TestBatch {
   }
 
   @Override
-  public String getTestModule() {
+  public String getTestModuleRelativeDir() {
     return moduleName;
   }
 
   @Override
+  public int getNumTestsInBatch() {
+    return tests.size();
+  }
+
+  @Override
   public int hashCode() {
     final int prime = 31;
     int result = 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestBatch.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestBatch.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestBatch.java
index 4ebb670..c537169 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestBatch.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestBatch.java
@@ -18,15 +18,31 @@
  */
 package org.apache.hive.ptest.execution.conf;
 
-public interface TestBatch {
+import java.util.concurrent.atomic.AtomicInteger;
 
-  public String getTestArguments();
-  
-  public String getTestClass();
+public abstract class TestBatch {
 
-  public String getName();
+  public TestBatch(AtomicInteger BATCH_ID_GEN) {
+    this.batchId = BATCH_ID_GEN.getAndIncrement();
+  }
 
-  public boolean isParallel();
+  private final int batchId;
+
+  public abstract String getTestArguments();
+
+  // TODO Get rid of this.
+  public abstract String getTestClass();
+
+  public abstract String getName();
+
+  public abstract boolean isParallel();
+
+  public abstract String getTestModuleRelativeDir();
+
+  public abstract int getNumTestsInBatch();
+
+  public final int getBatchId() {
+    return batchId;
+  }
 
-  public String getTestModule();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
index 2c5bd3a..f14026c 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestConfiguration.java
@@ -58,6 +58,7 @@ public class TestConfiguration {
   private static final String JENKINS_URL = "jenkinsURL";
   private static final String SSH_OPTS = "sshOpts";
   private static final String LOGS_URL = "logsURL";
+  // This ends up being set to "test" | mvn ${testCasePropertyName} for 
instance
   private static final String TEST_CASE_PROPERTY_NAME = "testCasePropertyName";
   private static final String BUILD_TOOL = "buildTool";
   // The following parameters are not supported yet. TODO Add support
@@ -226,14 +227,17 @@ public class TestConfiguration {
     return testCasePropertyName;
   }
 
+  // TODO Make sure this method is eventually used to find the prep / batch 
scripts.
   public String getApplyPathScriptPath() {
     return applyPathScriptPath;
   }
 
+  // TODO Make sure this method is eventually used to find the prep / batch 
scripts.
   public String getPrepTemplatePath() {
     return prepTemplatePath;
   }
 
+  // TODO Make sure this method is eventually used to find the prep / batch 
scripts.
   public String getBatchExecTemplatePath() {
     return batchExecTemplatePath;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java
index 5da804f..a243774 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java
@@ -24,16 +24,17 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,69 +45,44 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public class TestParser {
+
   private static final Splitter TEST_SPLITTER = Splitter.onPattern("[, ]")
-    .trimResults().omitEmptyStrings();
+      .trimResults().omitEmptyStrings();
 
   private static final String QTEST_MODULE_NAME = "itests/qtest";
   private static final String QTEST_SPARK_MODULE_NAME = "itests/qtest-spark";
 
+  private final AtomicInteger batchIdCounter;
+
   private final Context context;
   private final String testCasePropertyName;
   private final File sourceDirectory;
   private final Logger logger;
 
-  public TestParser(Context context, String testCasePropertyName, 
+  public TestParser(Context context, AtomicInteger batchIdCounter, String 
testCasePropertyName,
       File sourceDirectory, Logger logger) {
     this.context = context;
+    this.batchIdCounter = batchIdCounter;
     this.testCasePropertyName = testCasePropertyName;
     this.sourceDirectory = sourceDirectory;
     this.logger = logger;
   }
   private List<TestBatch> parseTests() {
-    Context unitContext = new Context(context.getSubProperties(
-        Joiner.on(".").join("unitTests", "")));
-    Set<String> excluded = 
Sets.newHashSet(TEST_SPLITTER.split(unitContext.getString("exclude", "")));
-    Set<String> isolated = 
Sets.newHashSet(TEST_SPLITTER.split(unitContext.getString("isolate", "")));
-    Set<String> included = 
Sets.newHashSet(TEST_SPLITTER.split(unitContext.getString("include", "")));
-    if(!included.isEmpty() && !excluded.isEmpty()) {
-      throw new IllegalArgumentException(String.format("Included and excluded 
mutally exclusive." +
-          " Included = %s, excluded = %s", included.toString(), 
excluded.toString()));
-    }
-    List<File> unitTestsDirs = Lists.newArrayList();
-    for(String unitTestDir : TEST_SPLITTER
-        .split(checkNotNull(unitContext.getString("directories"), 
"directories"))) {
-      File unitTestParent = new File(sourceDirectory, unitTestDir);
-      if(unitTestParent.isDirectory()) {
-        unitTestsDirs.add(unitTestParent);
-      } else {
-        logger.warn("Unit test directory " + unitTestParent + " does not 
exist.");
-      }
-    }
+
+    Set<String> excluded = new HashSet<String>();
+
+
     List<TestBatch> result = Lists.newArrayList();
     for(QFileTestBatch test : parseQFileTests()) {
       result.add(test);
       excluded.add(test.getDriver());
     }
-    for(File unitTestDir : unitTestsDirs) {
-      for(File classFile : FileUtils.listFiles(unitTestDir, new 
String[]{"class"}, true)) {
-        String className = classFile.getName();
-        logger.debug("In  " + unitTestDir  + ", found " + className);
-        if(className.startsWith("Test") && !className.contains("$")) {
-          String testName = className.replaceAll("\\.class$", "");
-          if(excluded.contains(testName)) {
-            logger.info("Exlcuding unit test " + testName);
-          } else if(included.isEmpty() || included.contains(testName)) {
-            if(isolated.contains(testName)) {
-              logger.info("Executing isolated unit test " + testName);
-              result.add(new UnitTestBatch(testCasePropertyName, testName, 
false));
-            } else {
-              logger.info("Executing parallel unit test " + testName);
-              result.add(new UnitTestBatch(testCasePropertyName, testName, 
true));
-            }
-          }
-        }
-      }
-    }
+
+    Collection<TestBatch> unitTestBatches =
+        new UnitTestPropertiesParser(context, batchIdCounter, 
testCasePropertyName, sourceDirectory, logger,
+            excluded).generateTestBatches();
+    result.addAll(unitTestBatches);
+
     return result;
   }
   private List<QFileTestBatch> parseQFileTests() {
@@ -185,11 +161,11 @@ public class TestParser {
         logger.info("Exlcuding test " + driver + " " + test);
       } else if(isolated.contains(test)) {
         logger.info("Executing isolated test " + driver + " " + test);
-        testBatches.add(new QFileTestBatch(testCasePropertyName, driver, 
queryFilesProperty,
+        testBatches.add(new QFileTestBatch(batchIdCounter, 
testCasePropertyName, driver, queryFilesProperty,
             Sets.newHashSet(test), isParallel, getModuleName(driver)));
       } else {
         if(testBatch.size() >= batchSize) {
-          testBatches.add(new QFileTestBatch(testCasePropertyName, driver, 
queryFilesProperty,
+          testBatches.add(new QFileTestBatch(batchIdCounter, 
testCasePropertyName, driver, queryFilesProperty,
               Sets.newHashSet(testBatch), isParallel, getModuleName(driver)));
           testBatch = Lists.newArrayList();
         }
@@ -197,7 +173,7 @@ public class TestParser {
       }
     }
     if(!testBatch.isEmpty()) {
-      testBatches.add(new QFileTestBatch(testCasePropertyName, driver, 
queryFilesProperty,
+      testBatches.add(new QFileTestBatch(batchIdCounter, testCasePropertyName, 
driver, queryFilesProperty,
           Sets.newHashSet(testBatch), isParallel, getModuleName(driver)));
     }
     return testBatches;
@@ -301,7 +277,7 @@ public class TestParser {
     File workingDir = new File("../..");
     File testConfigurationFile = new File(args[0]);
     TestConfiguration conf = TestConfiguration.fromFile(testConfigurationFile, 
log);
-    TestParser testParser = new TestParser(conf.getContext(), "test", 
workingDir, log);
+    TestParser testParser = new TestParser(conf.getContext(), new 
AtomicInteger(1), "test", workingDir, log);
     List<TestBatch> testBatches = testParser.parse().get();
     for (TestBatch testBatch : testBatches) {
       System.out.println(testBatch.getTestArguments());

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
index 51f7f90..cedc5a3 100644
--- 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestBatch.java
@@ -18,34 +18,58 @@
  */
 package org.apache.hive.ptest.execution.conf;
 
-public class UnitTestBatch implements TestBatch {
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+
+public class UnitTestBatch extends TestBatch {
 
   private final String testCasePropertyName;
-  private final String testName;
+  private final List<String> testList;
   private final boolean isParallel;
+  private final String moduleName;
+  private final String batchName;
 
-  public UnitTestBatch(String testCasePropertyName, String testName, boolean 
isParallel) {
+  public UnitTestBatch(AtomicInteger batchIdCounter, String 
testCasePropertyName,
+                       List<String> tests, String moduleName, boolean 
isParallel) {
+    super(batchIdCounter);
+    Preconditions.checkNotNull(testCasePropertyName);
+    Preconditions.checkArgument(tests!= null && !tests.isEmpty());
     this.testCasePropertyName = testCasePropertyName;
-    this.testName = testName;
+    this.testList = tests;
     this.isParallel = isParallel;
+    this.moduleName = moduleName;
+    if (tests.size() == 1) {
+      batchName = String.format("%d_%s", getBatchId(), tests.get(0));
+    } else {
+      batchName = String.format("%d_UTBatch_%s_%d_tests", getBatchId(),
+          (moduleName.replace("/", "__").replace(".", "__")), tests.size());
+    }
   }
   @Override
   public String getTestArguments() {
-    return String.format("-D%s=%s", testCasePropertyName, testName);
+    String testArg = Joiner.on(",").join(testList);
+    return String.format("-D%s=%s", testCasePropertyName, testArg);
   }
 
   @Override
   public String getName() {
-    return testName;
+    // Used for logDir, failure messages etc.
+    return batchName;
   }
   @Override
   public String getTestClass() {
-    return testName;
+    // Used to identify the module name. Return any.
+    return testList.get(0);
   }
+
   @Override
   public String toString() {
-    return "UnitTestBatch [testName=" + testName + ", isParallel=" + isParallel
-        + "]";
+    return "UnitTestBatch [name=" + batchName + ", id=" + getBatchId() + ", 
moduleName=" +
+        moduleName +", batchSize=" + testList.size() +
+        ", isParallel=" + isParallel + ", testList=" + testList + "]";
   }
   @Override
   public boolean isParallel() {
@@ -53,34 +77,45 @@ public class UnitTestBatch implements TestBatch {
   }
 
   @Override
-  public String getTestModule() {
-    return null;
+  public String getTestModuleRelativeDir() {
+    return moduleName;
   }
 
   @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + (isParallel ? 1231 : 1237);
-    result = prime * result + ((testName == null) ? 0 : testName.hashCode());
-    return result;
+  public int getNumTestsInBatch() {
+    return testList.size();
   }
+
   @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
+  public boolean equals(Object o) {
+    if (this == o) {
       return true;
-    if (obj == null)
+    }
+    if (o == null || getClass() != o.getClass()) {
       return false;
-    if (getClass() != obj.getClass())
+    }
+
+    UnitTestBatch that = (UnitTestBatch) o;
+
+    if (isParallel != that.isParallel) {
       return false;
-    UnitTestBatch other = (UnitTestBatch) obj;
-    if (isParallel != other.isParallel)
+    }
+    if (testList != null ? !testList.equals(that.testList) : that.testList != 
null) {
       return false;
-    if (testName == null) {
-      if (other.testName != null)
-        return false;
-    } else if (!testName.equals(other.testName))
+    }
+    if (moduleName != null ? !moduleName.equals(that.moduleName) : 
that.moduleName != null) {
       return false;
-    return true;
+    }
+    return batchName != null ? batchName.equals(that.batchName) : 
that.batchName == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = testList != null ? testList.hashCode() : 0;
+    result = 31 * result + (isParallel ? 1 : 0);
+    result = 31 * result + (moduleName != null ? moduleName.hashCode() : 0);
+    result = 31 * result + (batchName != null ? batchName.hashCode() : 0);
+    return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestPropertiesParser.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestPropertiesParser.java
 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestPropertiesParser.java
new file mode 100644
index 0000000..a482fce
--- /dev/null
+++ 
b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/UnitTestPropertiesParser.java
@@ -0,0 +1,666 @@
+/*
+ * 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.conf;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+
+class UnitTestPropertiesParser {
+
+  private static final Splitter VALUE_SPLITTER = Splitter.onPattern("[, ]")
+      .trimResults().omitEmptyStrings();
+
+  // Prefix for top level properties.
+  static final String PROP_PREFIX_ROOT = "unitTests";
+  // Prefix used to specify module specific properties. Mainly to avoid 
conflicts with older unitTests properties
+  static final String PROP_PREFIX_MODULE = "ut";
+
+  static final String PROP_DIRECTORIES = "directories";
+  static final String PROP_INCLUDE = "include";
+  static final String PROP_EXCLUDE = "exclude";
+  static final String PROP_ISOLATE = "isolate";
+  static final String PROP_SKIP_BATCHING = "skipBatching";
+  static final String PROP_BATCH_SIZE = "batchSize";
+  static final String PROP_SUBDIR_FOR_PREFIX = "subdirForPrefix";
+
+  static final String PROP_ONE_MODULE = "module";
+  static final String PROP_MODULE_LIST = "modules";
+
+  private final AtomicInteger batchIdCounter;
+
+  static final int DEFAULT_PROP_BATCH_SIZE = 1;
+  static final int DEFAULT_PROP_BATCH_SIZE_NOT_SPECIFIED = -1;
+  static final int DEFAULT_PROP_BATCH_SIZE_INCLUDE_ALL = 0;
+  static final String DEFAULT_PROP_DIRECTORIES = ".";
+  static final String DEFAULT_PROP_SUBDIR_FOR_PREFIX = "target";
+
+  static final String MODULE_NAME_TOP_LEVEL = "_root_"; // Special module for 
tests in the rootDir.
+  static final String PREFIX_TOP_LEVEL = ".";
+
+  private final Context unitRootContext; // Everything prefixed by ^unitTests.
+  private final Context unitModuleContext; // Everything prefixed by ^ut.
+  private final String testCasePropertyName;
+  private final Logger logger;
+  private final File sourceDirectory;
+  private final FileListProvider fileListProvider;
+  private final Set<String> excludedProvided; // excludedProvidedBy Framework 
vs excludedConfigured
+  private final boolean inTest;
+
+
+  @VisibleForTesting
+  UnitTestPropertiesParser(Context testContext, AtomicInteger batchIdCounter, 
String testCasePropertyName,
+                           File sourceDirectory, Logger logger,
+                           FileListProvider fileListProvider,
+                           Set<String> excludedProvided, boolean inTest) {
+    logger.info("{} created with sourceDirectory={}, testCasePropertyName={}, 
excludedProvide={}",
+        "fileListProvider={}, inTest={}",
+        UnitTestPropertiesParser.class.getSimpleName(), sourceDirectory, 
testCasePropertyName,
+        excludedProvided,
+        (fileListProvider == null ? "null" : 
fileListProvider.getClass().getSimpleName()), inTest);
+    Preconditions.checkNotNull(batchIdCounter, "batchIdCounter cannot be 
null");
+    Preconditions.checkNotNull(testContext, "testContext cannot be null");
+    Preconditions.checkNotNull(testCasePropertyName, "testCasePropertyName 
cannot be null");
+    Preconditions.checkNotNull(sourceDirectory, "sourceDirectory cannot be 
null");
+    Preconditions.checkNotNull(logger, "logger must be specified");
+    this.batchIdCounter = batchIdCounter;
+    this.unitRootContext =
+        new 
Context(testContext.getSubProperties(Joiner.on(".").join(PROP_PREFIX_ROOT, 
"")));
+    this.unitModuleContext =
+        new 
Context(testContext.getSubProperties(Joiner.on(".").join(PROP_PREFIX_MODULE, 
"")));
+    this.sourceDirectory = sourceDirectory;
+    this.testCasePropertyName = testCasePropertyName;
+    this.logger = logger;
+    if (excludedProvided != null) {
+      this.excludedProvided = excludedProvided;
+    } else {
+      this.excludedProvided = new HashSet<>();
+    }
+    if (fileListProvider != null) {
+      this.fileListProvider = fileListProvider;
+    } else {
+      this.fileListProvider = new DefaultFileListProvider();
+    }
+    this.inTest = inTest;
+
+  }
+
+  UnitTestPropertiesParser(Context testContext, AtomicInteger batchIdCounter, 
String testCasePropertyName,
+                           File sourceDirectory, Logger logger,
+                           Set<String> excludedProvided) {
+    this(testContext, batchIdCounter, testCasePropertyName, sourceDirectory, 
logger, null, excludedProvided, false);
+  }
+
+
+  Collection<TestBatch> generateTestBatches() {
+    try {
+      return parse();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+
+  private Collection<TestBatch> parse() throws IOException {
+
+    RootConfig rootConfig = getRootConfig(unitRootContext);
+    logger.info("RootConfig: " + rootConfig);
+
+    // TODO: Set this up as a tree, instead of a flat list.
+    Map<String, ModuleConfig> moduleConfigs = extractModuleConfigs();
+    logger.info("ModuleConfigs: {} ", moduleConfigs);
+
+    List<TestDir> unitTestsDirs = processPropertyDirectories();
+
+    validateConfigs(rootConfig, moduleConfigs, unitTestsDirs);
+
+    LinkedHashMap<String, LinkedHashSet<TestInfo>> allTests =
+        generateFullTestSet(rootConfig, moduleConfigs, unitTestsDirs);
+
+
+    return createTestBatches(allTests, rootConfig, moduleConfigs);
+  }
+
+  private Collection<TestBatch> createTestBatches(
+      LinkedHashMap<String, LinkedHashSet<TestInfo>> allTests, RootConfig 
rootConfig,
+      Map<String, ModuleConfig> moduleConfigs) {
+    List<TestBatch> testBatches = new LinkedList<>();
+    for (Map.Entry<String, LinkedHashSet<TestInfo>> entry : 
allTests.entrySet()) {
+      logger.info("Creating test batches for module={}, numTests={}", 
entry.getKey(),
+          entry.getValue().size());
+      String currentModule = entry.getKey();
+      String currentPathPrefix = getPathPrefixFromModuleName(currentModule);
+      int batchSize = rootConfig.batchSize;
+      if (moduleConfigs.containsKey(currentModule)) {
+        ModuleConfig moduleConfig = moduleConfigs.get(currentModule);
+        int batchSizeModule = moduleConfig.batchSize;
+        if (batchSizeModule != DEFAULT_PROP_BATCH_SIZE_NOT_SPECIFIED) {
+          batchSize = batchSizeModule;
+        }
+      }
+
+      if (batchSize == DEFAULT_PROP_BATCH_SIZE_INCLUDE_ALL) {
+        batchSize = Integer.MAX_VALUE;
+      }
+      logger.info("batchSize determined to be {} for module={}", batchSize, 
currentModule);
+
+      // TODO Even out the batch sizes (i.e. 20/20/1 should be replaced by 
14/14/13)
+      List<String> currentList = new LinkedList<>();
+      for (TestInfo testInfo : entry.getValue()) {
+        if (testInfo.isIsolated || testInfo.skipBatching) {
+          UnitTestBatch unitTestBatch =
+              new UnitTestBatch(batchIdCounter, testCasePropertyName, 
Collections.singletonList(testInfo.testName),
+                  currentPathPrefix, !testInfo.isIsolated);
+          testBatches.add(unitTestBatch);
+        } else {
+          currentList.add(testInfo.testName);
+          if (currentList.size() == batchSize) {
+            UnitTestBatch unitTestBatch =
+                new UnitTestBatch(batchIdCounter, testCasePropertyName, 
Collections.unmodifiableList(currentList),
+                    currentPathPrefix, true);
+            testBatches.add(unitTestBatch);
+            currentList = new LinkedList<>();
+          }
+        }
+      }
+      if (!currentList.isEmpty()) {
+        UnitTestBatch unitTestBatch =
+            new UnitTestBatch(batchIdCounter, testCasePropertyName, 
Collections.unmodifiableList(currentList),
+                currentPathPrefix, true);
+        testBatches.add(unitTestBatch);
+      }
+    }
+    return testBatches;
+  }
+
+
+  private RootConfig getRootConfig(Context context) {
+    ModuleConfig moduleConfig =
+        getModuleConfig(context, "irrelevant", DEFAULT_PROP_BATCH_SIZE);
+
+    String subDirForPrefix =
+        context.getString(PROP_SUBDIR_FOR_PREFIX, 
DEFAULT_PROP_SUBDIR_FOR_PREFIX);
+    Preconditions
+        .checkArgument(StringUtils.isNotBlank(subDirForPrefix) && 
!subDirForPrefix.contains("/"));
+
+    Context modulesContext =
+        new 
Context(context.getSubProperties(Joiner.on(".").join(PROP_MODULE_LIST, "")));
+    Set<String> includedModules = getProperty(modulesContext, PROP_INCLUDE);
+    Set<String> excludedModules = getProperty(modulesContext, PROP_EXCLUDE);
+    if (!includedModules.isEmpty() && !excludedModules.isEmpty()) {
+      throw new IllegalArgumentException(String.format(
+          "%s and %s are mutually exclusive for property %s. Provided values: 
included=%s, excluded=%s",
+          PROP_INCLUDE, PROP_EXCLUDE, PROP_MODULE_LIST, includedModules, 
excludedModules));
+    }
+
+    return new RootConfig(includedModules, excludedModules, 
moduleConfig.include,
+        moduleConfig.exclude, moduleConfig.skipBatching, moduleConfig.isolate,
+        moduleConfig.batchSize, subDirForPrefix);
+  }
+
+  private ModuleConfig getModuleConfig(Context context, String moduleName, int 
defaultBatchSize) {
+    Set<String> excluded = getProperty(context, PROP_EXCLUDE);
+    Set<String> isolated = getProperty(context, PROP_ISOLATE);
+    Set<String> included = getProperty(context, PROP_INCLUDE);
+    Set<String> skipBatching = getProperty(context, PROP_SKIP_BATCHING);
+    if (!included.isEmpty() && !excluded.isEmpty()) {
+      throw new IllegalArgumentException(String.format("Included and excluded 
mutually exclusive." +
+          " Included = %s, excluded = %s", included.toString(), 
excluded.toString()) +
+          " for module: " + moduleName);
+    }
+    int batchSize = context.getInteger(PROP_BATCH_SIZE, defaultBatchSize);
+
+    String pathPrefix = getPathPrefixFromModuleName(moduleName);
+
+    return new ModuleConfig(moduleName, included, excluded, skipBatching, 
isolated, batchSize,
+        pathPrefix);
+  }
+
+  private Set<String> getProperty(Context context, String propertyName) {
+    return 
Sets.newHashSet(VALUE_SPLITTER.split(context.getString(propertyName, "")));
+  }
+
+  private String getPathPrefixFromModuleName(String moduleName) {
+    String pathPrefix;
+    if (moduleName.equals(MODULE_NAME_TOP_LEVEL)) {
+      pathPrefix = PREFIX_TOP_LEVEL;
+    } else {
+      pathPrefix = moduleName.replace(".", "/");
+    }
+    return pathPrefix;
+  }
+
+  private String getModuleNameFromPathPrefix(String pathPrefix) {
+    if (pathPrefix.equals(PREFIX_TOP_LEVEL)) {
+      return MODULE_NAME_TOP_LEVEL;
+    } else {
+      pathPrefix = stripEndAndStart(pathPrefix, "/");
+      pathPrefix = pathPrefix.replace("/", ".");
+      // Example handling of dirs with a .
+      // shims/hadoop-2.6
+      //   -> moduleName=shims.hadoop-.2.6
+      return pathPrefix;
+    }
+  }
+
+  private String stripEndAndStart(String srcString, String stripChars) {
+    srcString = StringUtils.stripEnd(srcString, stripChars);
+    srcString = StringUtils.stripStart(srcString, stripChars);
+    return srcString;
+  }
+
+  private Map<String, ModuleConfig> extractModuleConfigs() {
+    Collection<String> modules = extractConfiguredModules();
+    Map<String, ModuleConfig> result = new HashMap<>();
+
+    for (String moduleName : modules) {
+      Context moduleContext =
+          new 
Context(unitModuleContext.getSubProperties(Joiner.on(".").join(moduleName, 
"")));
+      ModuleConfig moduleConfig =
+          getModuleConfig(moduleContext, moduleName, 
DEFAULT_PROP_BATCH_SIZE_NOT_SPECIFIED);
+      logger.info("Adding moduleConfig={}", moduleConfig);
+      result.put(moduleName, moduleConfig);
+    }
+    return result;
+  }
+
+  private Collection<String> extractConfiguredModules() {
+    List<String> configuredModules = new LinkedList<>();
+
+    Map<String, String> modulesMap = 
unitRootContext.getSubProperties(Joiner.on(".").join(
+        PROP_ONE_MODULE, ""));
+    for (Map.Entry<String, String> module : modulesMap.entrySet()) {
+      // This is an unnecessary check, and forced configuration in the 
property file. Maybe
+      // replace with an enforced empty value string.
+      Preconditions.checkArgument(module.getKey().equals(module.getValue()));
+      String moduleName = module.getKey();
+      configuredModules.add(moduleName);
+    }
+    return configuredModules;
+  }
+
+  private List<TestDir> processPropertyDirectories() throws IOException {
+    String srcDirString = sourceDirectory.getCanonicalPath();
+    List<TestDir> unitTestsDirs = Lists.newArrayList();
+    String propDirectoriies = unitRootContext.getString(PROP_DIRECTORIES, 
DEFAULT_PROP_DIRECTORIES);
+    Iterable<String> propDirectoriesIterable = 
VALUE_SPLITTER.split(propDirectoriies);
+
+    for (String unitTestDir : propDirectoriesIterable) {
+      File unitTestParent = new File(sourceDirectory, unitTestDir);
+      if (unitTestParent.isDirectory() || inTest) {
+        String absUnitTestDir = unitTestParent.getCanonicalPath();
+
+        Preconditions.checkState(absUnitTestDir.startsWith(srcDirString),
+            "Unit test dir: " + absUnitTestDir + " is not under provided src 
dir: " + srcDirString);
+        String modulePath = absUnitTestDir.substring(srcDirString.length());
+
+        modulePath = stripEndAndStart(modulePath, "/");
+
+        Preconditions.checkState(!modulePath.startsWith("/"),
+            String.format("Illegal module path: [%s]", modulePath));
+        if (StringUtils.isEmpty(modulePath)) {
+          modulePath = PREFIX_TOP_LEVEL;
+        }
+        String moduleName = getModuleNameFromPathPrefix(modulePath);
+        logger.info("modulePath determined as {} for testdir={}, 
DerivedModuleName={}", modulePath,
+            absUnitTestDir, moduleName);
+
+
+        logger.info("Adding unitTests dir [{}],[{}]", unitTestParent, 
moduleName);
+        unitTestsDirs.add(new TestDir(unitTestParent, moduleName));
+      } else {
+        logger.warn("Unit test directory " + unitTestParent + " does not 
exist, or is a file.");
+      }
+    }
+
+    return unitTestsDirs;
+  }
+
+  private void validateConfigs(RootConfig rootConfig,
+                               Map<String, ModuleConfig> moduleConfigs,
+                               List<TestDir> unitTestDir) {
+
+    if (rootConfig.include.isEmpty() & rootConfig.exclude.isEmpty()) {
+      // No conflicts. Module configuration is what will be used.
+      // We've already verified that includes and excludes are not present at 
the same time for
+      // individual modules.
+      return;
+    }
+
+    // Validate mainly for includes / excludes working as they should.
+    for (Map.Entry<String, ModuleConfig> entry : moduleConfigs.entrySet()) {
+      if (rootConfig.excludedModules.contains(entry.getKey())) {
+        // Don't bother validating.
+        continue;
+      }
+
+      if (!rootConfig.includedModules.isEmpty() &&
+          !rootConfig.includedModules.contains(entry.getKey())) {
+        // Include specified, but this module is not in the set.
+        continue;
+      }
+
+      // If global contains includes, individual modules can only contain 
additional includes.
+      if (!rootConfig.include.isEmpty() && 
!entry.getValue().exclude.isEmpty()) {
+        throw new IllegalStateException(String.format(
+            "Global config specified includes, while module config for %s 
specified excludes",
+            entry.getKey()));
+      }
+      // If global contains excludes, individual modules can only contain 
additional excludes.
+      if (!rootConfig.exclude.isEmpty() && 
!entry.getValue().include.isEmpty()) {
+        throw new IllegalStateException(String.format(
+            "Global config specified excludes, while module config for %s 
specified includes",
+            entry.getKey()));
+      }
+    }
+  }
+
+  private LinkedHashMap<String, LinkedHashSet<TestInfo>> 
generateFullTestSet(RootConfig rootConfig,
+                                                                             
Map<String, ModuleConfig> moduleConfigs,
+                                                                             
List<TestDir> unitTestDirs) throws
+      IOException {
+    LinkedHashMap<String, LinkedHashSet<TestInfo>> result = new 
LinkedHashMap<>();
+
+    for (TestDir unitTestDir : unitTestDirs) {
+      for (File classFile : fileListProvider
+          .listFiles(unitTestDir.path, new String[]{"class"}, true)) {
+        String className = classFile.getName();
+
+        if (className.startsWith("Test") && !className.contains("$")) {
+          String testName = className.replaceAll("\\.class$", "");
+          String pathPrefix = getPathPrefix(classFile, 
rootConfig.subDirForPrefix);
+          String moduleName = getModuleNameFromPathPrefix(pathPrefix);
+          logger.debug("In {}, found class {} with pathPrefix={}, 
moduleName={}", unitTestDir.path,
+              className,
+              pathPrefix, moduleName);
+
+
+          ModuleConfig moduleConfig = moduleConfigs.get(moduleName);
+          if (moduleConfig == null) {
+            moduleConfig = FAKE_MODULE_CONFIG;
+          }
+          TestInfo testInfo = checkAndGetTestInfo(moduleName, pathPrefix, 
testName, rootConfig, moduleConfig);
+          if (testInfo != null) {
+            logger.info("Adding test: " + testInfo);
+            addTestToResult(result, testInfo);
+          }
+        } else {
+          logger.trace("In {}, found class {} with pathPrefix={}. Not a test", 
unitTestDir.path,
+              className);
+        }
+      }
+    }
+    return result;
+  }
+
+  private void addTestToResult(Map<String, LinkedHashSet<TestInfo>> result, 
TestInfo testInfo) {
+    LinkedHashSet<TestInfo> moduleSet = result.get(testInfo.moduleName);
+    if (moduleSet == null) {
+      moduleSet = new LinkedHashSet<>();
+      result.put(testInfo.moduleName, moduleSet);
+    }
+    moduleSet.add(testInfo);
+  }
+
+  private String getPathPrefix(File file, String subDirPrefix) throws 
IOException {
+    String fname = file.getCanonicalPath();
+    
Preconditions.checkState(fname.startsWith(sourceDirectory.getCanonicalPath()));
+    fname = fname.substring(sourceDirectory.getCanonicalPath().length(), 
fname.length());
+    if (fname.contains(subDirPrefix)) {
+      fname = fname.substring(0, fname.indexOf(subDirPrefix));
+      fname = StringUtils.stripStart(fname, "/");
+      if (StringUtils.isEmpty(fname)) {
+        fname = PREFIX_TOP_LEVEL;
+      }
+      return fname;
+    } else {
+      logger.error("Could not find subDirPrefix {} in path: {}", subDirPrefix, 
fname);
+      return PREFIX_TOP_LEVEL;
+    }
+  }
+
+  private TestInfo checkAndGetTestInfo(String moduleName, String moduleRelDir, 
String testName,
+                                       RootConfig rootConfig, ModuleConfig 
moduleConfig) {
+    Preconditions.checkNotNull(moduleConfig);
+    TestInfo testInfo;
+    String rejectReason = null;
+    try {
+      if (rootConfig.excludedModules.contains(moduleName)) {
+        rejectReason = "root level module exclude";
+        return null;
+      }
+      if (!rootConfig.includedModules.isEmpty() &&
+          !rootConfig.includedModules.contains(moduleName)) {
+        rejectReason = "root level include, but not for module";
+        return null;
+      }
+      if (rootConfig.exclude.contains(testName)) {
+        rejectReason = "root excludes test";
+        return null;
+      }
+      if (moduleConfig.exclude.contains(testName)) {
+        rejectReason = "module excludes test";
+        return null;
+      }
+      boolean containsInclude = !rootConfig.include.isEmpty() || 
!moduleConfig.include.isEmpty();
+      if (containsInclude) {
+        if (!(rootConfig.include.contains(testName) || 
moduleConfig.include.contains(testName))) {
+          rejectReason = "test missing from include list";
+          return null;
+        }
+      }
+      if (excludedProvided.contains(testName)) {
+        // All qfiles handled via this...
+        rejectReason = "test present in provided exclude list";
+        return null;
+      }
+
+      // Add the test.
+      testInfo = new TestInfo(moduleName, moduleRelDir, testName, 
rootConfig.skipBatching.contains(testName) ||
+          moduleConfig.skipBatching.contains(testName),
+          rootConfig.isolate.contains(testName) || 
moduleConfig.isolate.contains(testName));
+      return testInfo;
+
+    } finally {
+      if (rejectReason != null) {
+        logger.debug("excluding {} due to {}", testName, rejectReason);
+      }
+    }
+  }
+
+  private static final class RootConfig {
+    private final Set<String> includedModules;
+    private final Set<String> excludedModules;
+    private final Set<String> include;
+    private final Set<String> exclude;
+    private final Set<String> skipBatching;
+    private final Set<String> isolate;
+    private final int batchSize;
+    private final String subDirForPrefix;
+
+    RootConfig(Set<String> includedModules, Set<String> excludedModules, 
Set<String> include,
+               Set<String> exclude, Set<String> skipBatching, Set<String> 
isolate,
+               int batchSize, String subDirForPrefix) {
+      this.includedModules = includedModules;
+      this.excludedModules = excludedModules;
+      this.include = include;
+      this.exclude = exclude;
+      this.skipBatching = skipBatching;
+      this.isolate = isolate;
+      this.batchSize = batchSize;
+      this.subDirForPrefix = subDirForPrefix;
+    }
+
+    @Override
+    public String toString() {
+      return "RootConfig{" +
+          "includedModules=" + includedModules +
+          ", excludedModules=" + excludedModules +
+          ", include=" + include +
+          ", exclude=" + exclude +
+          ", skipBatching=" + skipBatching +
+          ", isolate=" + isolate +
+          ", batchSize=" + batchSize +
+          ", subDirForPrefix='" + subDirForPrefix + '\'' +
+          '}';
+    }
+  }
+
+  private static final ModuleConfig FAKE_MODULE_CONFIG =
+      new ModuleConfig("_FAKE_", new HashSet<String>(), new HashSet<String>(),
+          new HashSet<String>(), new HashSet<String>(), 
DEFAULT_PROP_BATCH_SIZE_NOT_SPECIFIED,
+          "_fake_");
+
+  private static final class ModuleConfig {
+    private final String name;
+    private final Set<String> include;
+    private final Set<String> exclude;
+    private final Set<String> skipBatching;
+    private final Set<String> isolate;
+    private final String pathPrefix;
+    private final int batchSize;
+
+    ModuleConfig(String name, Set<String> include, Set<String> exclude,
+                 Set<String> skipBatching, Set<String> isolate, int batchSize,
+                 String pathPrefix) {
+      this.name = name;
+      this.include = include;
+      this.exclude = exclude;
+      this.skipBatching = skipBatching;
+      this.isolate = isolate;
+      this.batchSize = batchSize;
+      this.pathPrefix = pathPrefix;
+    }
+
+    @Override
+    public String toString() {
+      return "ModuleConfig{" +
+          "name='" + name + '\'' +
+          ", include=" + include +
+          ", exclude=" + exclude +
+          ", skipBatching=" + skipBatching +
+          ", isolate=" + isolate +
+          ", pathPrefix='" + pathPrefix + '\'' +
+          ", batchSize=" + batchSize +
+          '}';
+    }
+  }
+
+  private static class TestDir {
+    final File path;
+    final String module;
+
+    TestDir(File path, String module) {
+      this.path = path;
+      this.module = module;
+    }
+
+    @Override
+    public String toString() {
+      return "TestDir{" +
+          "path=" + path +
+          ", module='" + module + '\'' +
+          '}';
+    }
+  }
+
+  private static class TestInfo {
+    final String moduleName;
+    final String moduleRelativeDir;
+    final String testName;
+    final boolean skipBatching;
+    final boolean isIsolated;
+
+    TestInfo(String moduleName, String moduleRelativeDir, String testName, 
boolean skipBatching, boolean isIsolated) {
+      this.moduleName = moduleName;
+      this.moduleRelativeDir = moduleRelativeDir;
+      this.testName = testName;
+      this.skipBatching = skipBatching;
+      this.isIsolated = isIsolated;
+    }
+
+    @Override
+    public String toString() {
+      return "TestInfo{" +
+          "moduleName='" + moduleName + '\'' +
+          ", moduleRelativeDir='" + moduleRelativeDir + '\'' +
+          ", testName='" + testName + '\'' +
+          ", skipBatching=" + skipBatching +
+          ", isIsolated=" + isIsolated +
+          '}';
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      TestInfo testInfo = (TestInfo) o;
+
+      return skipBatching == testInfo.skipBatching && isIsolated == 
testInfo.isIsolated &&
+          moduleName.equals(testInfo.moduleName) &&
+          moduleRelativeDir.equals(testInfo.moduleRelativeDir) &&
+          testName.equals(testInfo.testName);
+
+    }
+
+    @Override
+    public int hashCode() {
+      int result = moduleName.hashCode();
+      result = 31 * result + moduleRelativeDir.hashCode();
+      result = 31 * result + testName.hashCode();
+      result = 31 * result + (skipBatching ? 1 : 0);
+      result = 31 * result + (isIsolated ? 1 : 0);
+      return result;
+    }
+  }
+
+  private static final class DefaultFileListProvider implements 
FileListProvider {
+
+    @Override
+    public Collection<File> listFiles(File directory, String[] extensions, 
boolean recursive) {
+      return FileUtils.listFiles(directory, extensions, recursive);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/resources/batch-exec.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/batch-exec.vm 
b/testutils/ptest2/src/main/resources/batch-exec.vm
index d8141b0..2cc56ea 100644
--- a/testutils/ptest2/src/main/resources/batch-exec.vm
+++ b/testutils/ptest2/src/main/resources/batch-exec.vm
@@ -16,6 +16,7 @@
 
 ##### Remember, this is a velocity template 
 set -x
+date +"%Y-%m-%d %T.%3N"
 umask 0022
 echo $$
 ps -e -o pid,pgrp,user,args
@@ -38,6 +39,8 @@ export M2_OPTS="-Xmx1g -XX:MaxPermSize=256m 
-Djava.io.tmpdir=$logDir/tmp ${maven
 export HADOOP_ROOT_LOGGER=INFO,console
 export HADOOP_OPTS="-Dhive.log.dir=$logDir -Dhive.query.id=hadoop 
-Djava.io.tmpdir=$logDir/tmp"
 cd $localDir/$instanceName/${repositoryName}-source || exit 1
+date +"%Y-%m-%d %T.%3N"
+echo "Pre test cleanup"
 if [[ -s batch.pid ]]
 then
   while read pid
@@ -53,10 +56,11 @@ echo "$$" > batch.pid
 find ./ -name 'TEST-*.xml' -delete
 find ./ -name 'hive.log' -delete
 find ./ -name junit_metastore_db | xargs -r rm -rf
+date +"%Y-%m-%d %T.%3N"
+echo "Pre test cleanup done"
 ret=0
 if [[ "${buildTool}" == "maven" ]]
 then
-  testModule=$(find ./ -name '${testClass}.java' | awk -F'/' '{print $2}')
   if [[ -z "$testModule" ]]
   then
     testModule=./
@@ -64,6 +68,7 @@ then
   pushd $testModule
   timeout 40m mvn -B test -Dmaven.repo.local=$localDir/$instanceName/maven \
     $mavenArgs $mavenTestArgs $testArguments 1>$logDir/maven-test.txt 2>&1 
</dev/null &
+  date +"%Y-%m-%d %T.%3N"
 #[[
   pid=$!
 ]]#
@@ -86,6 +91,7 @@ fi
 echo $pid >> batch.pid
 wait $pid
 ret=$?
+date +"%Y-%m-%d %T.%3N"
 find ./ -type f -name hive.log -o -name spark.log -o -name derby.log | \
   xargs -I {} sh -c 'f=$(basename {}); test -f ${logDir}/$f && 
f=$f-$(uuidgen); mv {} ${logDir}/$f'
 find ./ -type f -name 'TEST-*.xml' | \
@@ -94,7 +100,7 @@ find ./ -path "*/spark/work" | \
   xargs -I {} sh -c 'mv {} ${logDir}/spark-log'
 find ./ -type f -name 'syslog*' | \
   xargs -I {} sh -c 'mkdir -p ${logDir}/syslogs; mv {} ${logDir}/syslogs'
-
+date +"%Y-%m-%d %T.%3N"
 
 if [[ -f $logDir/.log ]]
 then

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/main/resources/source-prep.vm
----------------------------------------------------------------------
diff --git a/testutils/ptest2/src/main/resources/source-prep.vm 
b/testutils/ptest2/src/main/resources/source-prep.vm
index 9c83a14..67e6a95 100644
--- a/testutils/ptest2/src/main/resources/source-prep.vm
+++ b/testutils/ptest2/src/main/resources/source-prep.vm
@@ -16,6 +16,7 @@
 ##### Remember this is a velocity template
 set -e
 set -x
+date +"%Y-%m-%d %T.%3N"
 if [[ -n "${javaHome}" ]]
 then
   export JAVA_HOME=$javaHome
@@ -64,13 +65,15 @@ cd $workingDir/
     then
       git clone $repository ${repositoryName}-source
     fi
+    date +"%Y-%m-%d %T.%3N"
     cd ${repositoryName}-source
     git fetch origin
     git reset --hard HEAD && git clean -f -d
     git checkout $branch || git checkout -b $branch origin/$branch
     git reset --hard origin/$branch
     git merge --ff-only origin/$branch
-    git gc
+    #git gc
+    date +"%Y-%m-%d %T.%3N"
   else
     echo "Unknown repository type '${repositoryType}'"
     exit 1
@@ -91,14 +94,20 @@ cd $workingDir/
     for i in $(echo $ADDITIONAL_PROFILES | tr "," "\n")
       do
         mvn clean install -DskipTests -P$i;
-        cd itests
-        mvn clean install -DskipTests -P$i;
+        if [[ "-d itests" ]]
+        then
+          cd itests
+          mvn clean install -DskipTests -P$i;
         cd ..
+        fi
       done
     #end
     mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven 
$mavenArgs $mavenBuildArgs
-    cd itests
-    mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven 
$mavenArgs $mavenBuildArgs
+    if [[ -d "itests" ]]
+    then
+      cd itests
+      mvn -B clean install -DskipTests -Dmaven.repo.local=$workingDir/maven 
$mavenArgs $mavenBuildArgs
+    fi
   elif [[ "${buildTool}" == "ant" ]]
   then
     ant $antArgs -Divy.default.ivy.user.dir=$workingDir/ivy \
@@ -108,5 +117,6 @@ cd $workingDir/
      echo "Unknown build tool ${buildTool}"
      exit 127
    fi
+   date +"%Y-%m-%d %T.%3N"
 ) 2>&1 | tee $logDir/source-prep.txt
 exit ${PIPESTATUS[0]}

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java
index 6347ce5..3906435 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockRSyncCommandExecutor.java
@@ -22,6 +22,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hive.ptest.execution.ssh.RSyncCommand;
 import org.apache.hive.ptest.execution.ssh.RSyncCommandExecutor;
@@ -33,6 +35,7 @@ import com.google.common.collect.Maps;
 public class MockRSyncCommandExecutor extends RSyncCommandExecutor {
   private final List<String> mCommands;
   private final Map<String, Queue<Integer>> mFailures;
+  private final AtomicInteger matchCount = new AtomicInteger(0);
   public MockRSyncCommandExecutor(Logger logger) {
     super(logger, 0, null);
     mCommands = Lists.newArrayList();
@@ -62,9 +65,13 @@ public class MockRSyncCommandExecutor extends 
RSyncCommandExecutor {
     if(queue == null || queue.isEmpty()) {
       command.setExitCode(0);
     } else {
+      matchCount.incrementAndGet();
       command.setExitCode(queue.remove());
     }
   }
 
+  public int getMatchCount() {
+    return matchCount.get();
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java
index e4cd807..1f3db12 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/MockSSHCommandExecutor.java
@@ -22,6 +22,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hive.ptest.execution.ssh.SSHCommand;
 import org.apache.hive.ptest.execution.ssh.SSHCommandExecutor;
@@ -33,6 +34,7 @@ import com.google.common.collect.Maps;
 public class MockSSHCommandExecutor extends SSHCommandExecutor {
   private final List<String> mCommands;
   private final Map<String, Queue<Integer>> mFailures;
+  private final AtomicInteger matchCount = new AtomicInteger(0);
   public MockSSHCommandExecutor(Logger logger) {
     super(logger);
     mCommands = Lists.newArrayList();
@@ -61,7 +63,12 @@ public class MockSSHCommandExecutor extends 
SSHCommandExecutor {
     if(queue == null || queue.isEmpty()) {
       command.setExitCode(0);
     } else {
+      matchCount.incrementAndGet();
       command.setExitCode(queue.remove());
     }
   }
+
+  public int getMatchCount() {
+    return matchCount.get();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
index 29d148b..bb1bb3e 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.java
@@ -20,9 +20,11 @@ package org.apache.hive.ptest.execution;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hive.ptest.execution.conf.QFileTestBatch;
@@ -70,12 +72,12 @@ public class TestExecutionPhase extends AbstractTestPhase {
     testDir = Dirs.create( new File(baseDir, "test"));
     Assert.assertTrue(new File(testDir, QFILENAME).createNewFile());
     testBatch =
-        new QFileTestBatch("testcase", DRIVER, "qfile", 
Sets.newHashSet(QFILENAME), isParallel,
+        new QFileTestBatch(new AtomicInteger(1), "testcase", DRIVER, "qfile", 
Sets.newHashSet(QFILENAME), isParallel,
             "testModule");
     testBatches = Collections.singletonList(testBatch);
   }
   private void setupUnitTest() throws Exception {
-    testBatch = new UnitTestBatch("testcase", DRIVER, false);
+    testBatch = new UnitTestBatch(new AtomicInteger(1), "testcase", 
Arrays.asList(DRIVER), "fakemodule", false);
     testBatches = Collections.singletonList(testBatch);
   }
   private void copyTestOutput(String resource, File directory, String name) 
throws Exception {
@@ -104,6 +106,7 @@ public class TestExecutionPhase extends AbstractTestPhase {
         "-0/scratch/hiveptest-" + DRIVER + "-" + QFILENAME + ".sh", 1);
     copyTestOutput("SomeTest-failure.xml", failedLogDir, testBatch.getName());
     getPhase().execute();
+    Assert.assertEquals(1, sshCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), 
executedTests);
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), failedTests);
@@ -121,9 +124,10 @@ public class TestExecutionPhase extends AbstractTestPhase {
   public void testFailingUnitTest() throws Throwable {
     setupUnitTest();
     sshCommandExecutor.putFailure("bash " + LOCAL_DIR + "/" + HOST + "-" + 
USER +
-        "-0/scratch/hiveptest-" + DRIVER + ".sh", 1);
+        "-0/scratch/hiveptest-" + testBatch.getBatchId() + "_" + DRIVER + 
".sh", 1);
     copyTestOutput("SomeTest-failure.xml", failedLogDir, testBatch.getName());
     getPhase().execute();
+    Assert.assertEquals(1, sshCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), 
executedTests);
     Assert.assertEquals(Sets.newHashSet("SomeTest." + QFILENAME), failedTests);

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt
index 0727830..97b66af 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testFailingUnitTest.approved.txt
@@ -1,9 +1,9 @@
 /some/working/dir/ivy /some/local/dir/somehost-someuser-0
 /some/working/dir/maven /some/local/dir/somehost-someuser-0
 /some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0
-/tmp/hive-ptest-units/TestExecutionPhase/logs/failed/driver 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh
+/tmp/hive-ptest-units/TestExecutionPhase/logs/failed/1_driver 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-1_driver.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver.sh
+bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver.sh
 killall -q -9 -f java || true
 mkdir -p /some/local/dir/somehost-someuser-0/logs 
/some/local/dir/somehost-someuser-0/maven 
/some/local/dir/somehost-someuser-0/scratch 
/some/local/dir/somehost-someuser-0/ivy 
/some/local/dir/somehost-someuser-0/repositoryName-source
 mkdir -p /some/local/dir/somehost-someuser-1/logs 
/some/local/dir/somehost-someuser-1/maven 
/some/local/dir/somehost-someuser-1/scratch 
/some/local/dir/somehost-someuser-1/ivy 
/some/local/dir/somehost-someuser-1/repositoryName-source

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt
index 3ce10b1..9cb4715 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestExecutionPhase.testPassingUnitTest.approved.txt
@@ -1,9 +1,9 @@
 /some/working/dir/ivy /some/local/dir/somehost-someuser-0
 /some/working/dir/maven /some/local/dir/somehost-someuser-0
 /some/working/dir/repositoryName-source /some/local/dir/somehost-someuser-0
-/tmp/hive-ptest-units/TestExecutionPhase/logs/succeeded/driver 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-driver.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver.sh
+/tmp/hive-ptest-units/TestExecutionPhase/logs/succeeded/1_driver 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestExecutionPhase/scratch/hiveptest-1_driver.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver.sh
+bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver.sh
 killall -q -9 -f java || true
 mkdir -p /some/local/dir/somehost-someuser-0/logs 
/some/local/dir/somehost-someuser-0/maven 
/some/local/dir/somehost-someuser-0/scratch 
/some/local/dir/somehost-someuser-0/ivy 
/some/local/dir/somehost-someuser-0/repositoryName-source
 mkdir -p /some/local/dir/somehost-someuser-1/logs 
/some/local/dir/somehost-someuser-1/maven 
/some/local/dir/somehost-someuser-1/scratch 
/some/local/dir/somehost-someuser-1/ivy 
/some/local/dir/somehost-someuser-1/repositoryName-source

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
index 0acebb9..65cf6a0 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.java
@@ -22,12 +22,14 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import junit.framework.Assert;
 
@@ -104,10 +106,11 @@ public class TestHostExecutor {
     parallelWorkQueue = new LinkedBlockingQueue<TestBatch>();
     isolatedWorkQueue = new LinkedBlockingQueue<TestBatch>();
     failedTestResults = Sets.newHashSet();
-    testBatchParallel1 = new UnitTestBatch("testcase", DRIVER_PARALLEL_1, 
true);
-    testBatchParallel2 = new UnitTestBatch("testcase", DRIVER_PARALLEL_2, 
true);
-    testBatchIsolated1 = new UnitTestBatch("testcase", DRIVER_ISOLATED_1, 
false);
-    testBatchIsolated2 = new UnitTestBatch("testcase", DRIVER_ISOLATED_2, 
false);
+    AtomicInteger unitTestBatchCounter = new AtomicInteger(1);
+    testBatchParallel1 = new UnitTestBatch(unitTestBatchCounter, "testcase", 
Arrays.asList(DRIVER_PARALLEL_1), "fakeModule1", true);
+    testBatchParallel2 = new UnitTestBatch(unitTestBatchCounter, "testcase", 
Arrays.asList(DRIVER_PARALLEL_2), "fakeModule2", true);
+    testBatchIsolated1 = new UnitTestBatch(unitTestBatchCounter, "testcase", 
Arrays.asList(DRIVER_ISOLATED_1), "fakeModule3", false);
+    testBatchIsolated2 = new UnitTestBatch(unitTestBatchCounter, "testcase", 
Arrays.asList(DRIVER_ISOLATED_2), "fakeModule4", false);
     executor = 
MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(2));
     localCommandFactory = new MockLocalCommandFactory(LOG);
     localCommand = mock(LocalCommand.class);
@@ -159,44 +162,55 @@ public class TestHostExecutor {
   @Test
   public void testParallelFailsOnExec()
       throws Exception {
-    sshCommandExecutor.putFailure("bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh",
+    sshCommandExecutor.putFailure("bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-" +
+            testBatchParallel1.getBatchId() + "_driver-parallel-1.sh",
         Constants.EXIT_CODE_UNKNOWN);
     HostExecutor executor = createHostExecutor();
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, 
failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(parallelWorkQueue.toString(), 
parallelWorkQueue.isEmpty());
+    Assert.assertEquals(1, sshCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
   }
   @Test
   public void testIsolatedFailsOnExec()
       throws Exception {
-    sshCommandExecutor.putFailure("bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh",
+    sshCommandExecutor.putFailure("bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-" +
+            testBatchIsolated1.getBatchId() + "_driver-isolated-1.sh",
         Constants.EXIT_CODE_UNKNOWN);
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, 
failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), 
parallelWorkQueue.isEmpty());
+    Assert.assertEquals(1, sshCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
   }
   @Test
   public void testParallelFailsOnRsync()
       throws Exception {
-    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-1.sh
 "
-        + 
"/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh", 
Constants.EXIT_CODE_UNKNOWN);
+    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-"
 +
+            testBatchParallel1.getBatchId() + "_driver-parallel-1.sh "
+            + "/some/local/dir/somehost-someuser-0/scratch/hiveptest-" +
+            testBatchParallel1.getBatchId() + "_driver-parallel-1.sh",
+        Constants.EXIT_CODE_UNKNOWN);
     HostExecutor executor = createHostExecutor();
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, 
failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(parallelWorkQueue.toString(), 
parallelWorkQueue.isEmpty());
+    Assert.assertEquals(1, rsyncCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
   }
   @Test
   public void testShutdownBeforeExec()
       throws Exception {
-    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-1.sh
 "
-        + 
"/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh", 
Constants.EXIT_CODE_UNKNOWN);
+    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-"
 +
+            testBatchParallel1.getBatchId() + "_driver-parallel-1.sh "
+            + "/some/local/dir/somehost-someuser-0/scratch/hiveptest-" +
+            testBatchParallel1.getBatchId() + "_driver-parallel-1.sh",
+        Constants.EXIT_CODE_UNKNOWN);
     HostExecutor executor = createHostExecutor();
     parallelWorkQueue.addAll(Lists.newArrayList(testBatchParallel1));
     executor.shutdownNow();
@@ -204,30 +218,38 @@ public class TestHostExecutor {
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertEquals(parallelWorkQueue.toString(), 1, 
parallelWorkQueue.size());
     Approvals.verify("EMPTY\n" + getExecutedCommands());
+    Assert.assertEquals(0, rsyncCommandExecutor.getMatchCount());
     Assert.assertTrue(executor.isShutdown());
   }
   @Test
   public void testIsolatedFailsOnRsyncUnknown()
       throws Exception {
-    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh
 "+
-        
"/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh", 
Constants.EXIT_CODE_UNKNOWN);
+    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-"
 +
+            testBatchIsolated1.getBatchId() + "_driver-isolated-1.sh " +
+            "/some/local/dir/somehost-someuser-0/scratch/hiveptest-" +
+            testBatchIsolated1.getBatchId() + "_driver-isolated-1.sh",
+        Constants.EXIT_CODE_UNKNOWN);
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, 
failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), 
isolatedWorkQueue.isEmpty());
+    Assert.assertEquals(1, rsyncCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
   }
   @Test
   public void testIsolatedFailsOnRsyncOne()
       throws Exception {
-    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh
 "+
-        
"/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh", 
1);
+    
rsyncCommandExecutor.putFailure("/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-"
 +
+        testBatchIsolated1.getBatchId() + "_driver-isolated-1.sh " +
+        "/some/local/dir/somehost-someuser-0/scratch/hiveptest-" + 
testBatchIsolated1.getBatchId() +
+        "_driver-isolated-1.sh", 1);
     HostExecutor executor = createHostExecutor();
     isolatedWorkQueue.addAll(Lists.newArrayList(testBatchIsolated1));
     executor.submitTests(parallelWorkQueue, isolatedWorkQueue, 
failedTestResults).get();
     Assert.assertEquals(Collections.emptySet(),  failedTestResults);
     Assert.assertTrue(isolatedWorkQueue.toString(), 
parallelWorkQueue.isEmpty());
+    Assert.assertEquals(1, rsyncCommandExecutor.getMatchCount());
     Approvals.verify(getExecutedCommands());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testBasic.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testBasic.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testBasic.approved.txt
index c2a702c..c4cc9f6 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testBasic.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testBasic.approved.txt
@@ -1,12 +1,12 @@
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-isolated-1 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-isolated-2 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-parallel-1 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-parallel-2 
/some/local/dir/somehost-someuser-0/logs/
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-2.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-2.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-1.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-parallel-2.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-2.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-2.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-1.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-parallel-2.sh
\ No newline at end of file
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/1_driver-parallel-1 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/2_driver-parallel-2 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/3_driver-isolated-1 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/4_driver-isolated-2 
/some/local/dir/somehost-someuser-0/logs/
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-1_driver-parallel-1.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver-parallel-1.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-2_driver-parallel-2.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-2_driver-parallel-2.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-4_driver-isolated-2.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-4_driver-isolated-2.sh
+bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-1_driver-parallel-1.sh
+bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-2_driver-parallel-2.sh
+bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-4_driver-isolated-2.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnExec.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnExec.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnExec.approved.txt
index 2a3a4a6..506b2e0 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnExec.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnExec.approved.txt
@@ -1,5 +1,5 @@
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
-bash /some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-bash /some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
\ No newline at end of file
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/3_driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
+bash 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+bash 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncOne.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncOne.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncOne.approved.txt
index 13945bf..a460175 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncOne.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncOne.approved.txt
@@ -1,4 +1,4 @@
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
-bash /some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
\ No newline at end of file
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/3_driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
+bash 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/eb126207/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncUnknown.approved.txt
----------------------------------------------------------------------
diff --git 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncUnknown.approved.txt
 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncUnknown.approved.txt
index 13945bf..a460175 100644
--- 
a/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncUnknown.approved.txt
+++ 
b/testutils/ptest2/src/test/java/org/apache/hive/ptest/execution/TestHostExecutor.testIsolatedFailsOnRsyncUnknown.approved.txt
@@ -1,4 +1,4 @@
-/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-0/scratch/hiveptest-driver-isolated-1.sh
-/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-driver-isolated-1.sh 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
-bash /some/local/dir/somehost-someuser-1/scratch/hiveptest-driver-isolated-1.sh
\ No newline at end of file
+/tmp/hive-ptest-units/TestHostExecutor/logs/succeeded/3_driver-isolated-1 
/some/local/dir/somehost-someuser-1/logs/
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-0/scratch/hiveptest-3_driver-isolated-1.sh
+/tmp/hive-ptest-units/TestHostExecutor/scratch/hiveptest-3_driver-isolated-1.sh
 /some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
+bash 
/some/local/dir/somehost-someuser-1/scratch/hiveptest-3_driver-isolated-1.sh
\ No newline at end of file

Reply via email to