You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2015/09/30 17:41:14 UTC

[14/58] [abbrv] hadoop git commit: HADOOP-11984. Enable parallel JUnit tests in pre-commit (Contributed by Chris Nauroth)

HADOOP-11984. Enable parallel JUnit tests in pre-commit (Contributed by Chris Nauroth)


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

Branch: refs/heads/HDFS-7285
Commit: f0f984e4e63d0dbafe93062a122ee051330db301
Parents: 0b31c23
Author: Vinayakumar B <vi...@apache.org>
Authored: Sun Sep 27 14:12:07 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Sun Sep 27 14:12:07 2015 +0530

----------------------------------------------------------------------
 dev-support/test-patch.sh                       | 29 +++++++++-
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 hadoop-common-project/hadoop-common/pom.xml     | 43 +++++++++++++-
 .../apache/hadoop/conf/TestConfiguration.java   |  2 +-
 .../apache/hadoop/ha/ClientBaseWithFixes.java   | 40 ++-----------
 .../http/TestAuthenticationSessionCookie.java   |  5 +-
 .../apache/hadoop/http/TestHttpCookieFlag.java  |  7 +--
 .../apache/hadoop/http/TestSSLHttpServer.java   |  4 +-
 .../org/apache/hadoop/net/ServerSocketUtil.java |  7 ++-
 .../hadoop/security/ssl/KeyStoreTestUtil.java   | 60 ++++++++++++++++++--
 hadoop-project/pom.xml                          |  2 +-
 11 files changed, 142 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
index a3cdc85..23b2272 100755
--- a/dev-support/test-patch.sh
+++ b/dev-support/test-patch.sh
@@ -599,6 +599,8 @@ function hadoop_usage
   echo "--run-tests            Run all relevant tests below the base directory"
   echo "--skip-system-plugins  Do not load plugins from ${BINDIR}/test-patch.d"
   echo "--testlist=<list>      Specify which subsystem tests to use (comma delimited)"
+  echo "--test-parallel=<bool> Run multiple tests in parallel (default false in developer mode, true in Jenkins mode)"
+  echo "--test-threads=<int>   Number of tests to run in parallel (default defined in ${PROJECT_NAME} build)"
 
   echo "Shell binary overrides:"
   echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
@@ -691,6 +693,7 @@ function parse_args
       ;;
       --jenkins)
         JENKINS=true
+        TEST_PARALLEL=${TEST_PARALLEL:-true}
       ;;
       --jira-cmd=*)
         JIRACLI=${i#*=}
@@ -749,6 +752,12 @@ function parse_args
           add_test "${j}"
         done
       ;;
+      --test-parallel=*)
+        TEST_PARALLEL=${i#*=}
+      ;;
+      --test-threads=*)
+        TEST_THREADS=${i#*=}
+      ;;
       --wget-cmd=*)
         WGET=${i#*=}
       ;;
@@ -811,6 +820,13 @@ function parse_args
   PATCH_DIR=$(cd -P -- "${PATCH_DIR}" >/dev/null && pwd -P)
 
   GITDIFFLINES=${PATCH_DIR}/gitdifflines.txt
+
+  if [[ ${TEST_PARALLEL} == "true" ]] ; then
+    PARALLEL_TESTS_PROFILE=-Pparallel-tests
+    if [[ -n ${TEST_THREADS:-} ]]; then
+      TESTS_THREAD_COUNT="-DtestsThreadCount=$TEST_THREADS"
+    fi
+  fi
 }
 
 ## @description  Locate the pom.xml file for a given directory
@@ -2245,13 +2261,22 @@ function check_unittests
 
     test_logfile=${PATCH_DIR}/testrun_${module_suffix}.txt
     echo "  Running tests in ${module_suffix}"
-    echo_and_redirect "${test_logfile}" "${MVN}" "${MAVEN_ARGS[@]}" clean install -fae ${NATIVE_PROFILE} ${REQUIRE_TEST_LIB_HADOOP} -D${PROJECT_NAME}PatchProcess
+    # Temporary hack to run the parallel tests profile only for hadoop-common.
+    # This code will be removed once hadoop-hdfs is ready for parallel test
+    # execution.
+    if [[ ${module} == "hadoop-common-project/hadoop-common" ]] ; then
+      OPTIONAL_PARALLEL_TESTS_PROFILE=${PARALLEL_TESTS_PROFILE}
+    else
+      unset OPTIONAL_PARALLEL_TESTS_PROFILE
+    fi
+    # shellcheck disable=2086
+    echo_and_redirect "${test_logfile}" "${MVN}" "${MAVEN_ARGS[@]}" clean install -fae ${NATIVE_PROFILE} ${REQUIRE_TEST_LIB_HADOOP} ${OPTIONAL_PARALLEL_TESTS_PROFILE} ${TESTS_THREAD_COUNT} -D${PROJECT_NAME}PatchProcess
     test_build_result=$?
 
     add_jira_footer "${module_suffix} test log" "@@BASE@@/testrun_${module_suffix}.txt"
 
     # shellcheck disable=2016
-    module_test_timeouts=$(${AWK} '/^Running / { if (last) { print last } last=$2 } /^Tests run: / { last="" }' "${test_logfile}")
+    module_test_timeouts=$(${AWK} '/^Running / { array[$NF] = 1 } /^Tests run: .* in / { delete array[$NF] } END { for (x in array) { print x } }' "${test_logfile}")
     if [[ -n "${module_test_timeouts}" ]] ; then
       test_timeouts="${test_timeouts} ${module_test_timeouts}"
       result=1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 77929dc..c7c5de2 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -822,6 +822,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11878. FileContext#fixRelativePart should check for not null for a
     more informative exception. (Brahma Reddy Battula via kasha)
 
+    HADOOP-11984. Enable parallel JUnit tests in pre-commit.
+    (Chris Nauroth via vinayakumarb)
+
   BUG FIXES
 
     HADOOP-12374. Updated expunge command description.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 3ae09a0..ef77dbd 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -879,11 +879,52 @@
       <build>
         <plugins>
           <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <script language="javascript"><![CDATA[
+                      var baseDirs = [
+                          "${test.build.data}",
+                          "${test.build.dir}",
+                          "${hadoop.tmp.dir}" ];
+                      for (var i in baseDirs) {
+                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
+                          var mkdir = project.createTask("mkdir");
+                          mkdir.setDir(new java.io.File(baseDirs[i], j));
+                          mkdir.perform();
+                        }
+                      }
+                    ]]></script>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-surefire-plugin</artifactId>
             <configuration>
               <forkCount>${testsThreadCount}</forkCount>
-              <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError -DminiClusterDedicatedDirs=true</argLine>
+              <reuseForks>false</reuseForks>
+              <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+              <systemPropertyVariables>
+                <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+                <!-- Due to a Maven quirk, setting this to just -->
+                <!-- surefire.forkNumber won't do the parameter substitution. -->
+                <!-- Putting a prefix in front of it like "fork-" makes it -->
+                <!-- work. -->
+                <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+              </systemPropertyVariables>
             </configuration>
           </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index a039741..46b2e50 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -1493,7 +1493,7 @@ public class TestConfiguration extends TestCase {
 
       @Override
       public void run() {
-        for (int i = 0; i < 100000; i++) {
+        for (int i = 0; i < 10000; i++) {
           config.set("some.config.value-" + prefix + i, "value");
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
index 5f03133..b1ce1d1 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
@@ -23,10 +23,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
-import java.io.RandomAccessFile;
 import java.net.Socket;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -34,8 +31,8 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.util.Time;
-import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -167,10 +164,6 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
     private LinkedList<ZooKeeper> allClients;
     private boolean allClientsSetup = false;
 
-    private RandomAccessFile portNumLockFile;
-
-    private File portNumFile;
-
     protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp)
         throws IOException, InterruptedException
     {
@@ -413,29 +406,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
 
     private String initHostPort() {
         BASETEST.mkdirs();
-        int port;
-        for (;;) {
-            port = PortAssignment.unique();
-            FileLock lock = null;
-            portNumLockFile = null;
-            try {
-                try {
-                    portNumFile = new File(BASETEST, port + ".lock");
-                    portNumLockFile = new RandomAccessFile(portNumFile, "rw");
-                    try {
-                        lock = portNumLockFile.getChannel().tryLock();
-                    } catch (OverlappingFileLockException e) {
-                        continue;
-                    }
-                } finally {
-                    if (lock != null)
-                        break;
-                    if (portNumLockFile != null)
-                        portNumLockFile.close();
-                }
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+        int port = 0;
+        try {
+           port = ServerSocketUtil.getPort(port, 100);
+        } catch (IOException e) {
+           throw new RuntimeException(e);
         }
         return "127.0.0.1:" + port;
     }
@@ -480,9 +455,6 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
 
         stopServer();
 
-        portNumLockFile.close();
-        portNumFile.delete();
-        
         if (tmpDir != null) {
             Assert.assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
index e435034..058633a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
@@ -113,10 +113,7 @@ public class TestAuthenticationSessionCookie {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
-
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     server = new HttpServer2.Builder()
             .setName("test")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
index 5c5ed48..f73b019 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
@@ -17,7 +17,6 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.apache.hadoop.security.ssl.SSLFactory;
@@ -27,12 +26,10 @@ import org.junit.Test;
 
 import javax.net.ssl.HttpsURLConnection;
 import javax.servlet.*;
-import javax.servlet.http.Cookie;
 import javax.servlet.http.HttpServletResponse;
 import java.io.File;
 import java.io.IOException;
 import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.security.GeneralSecurityException;
@@ -89,9 +86,7 @@ public class TestHttpCookieFlag {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
     clientSslFactory.init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
index 70fea87..1371964 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
@@ -65,9 +65,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
     sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
 
     KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-    Configuration sslConf = new Configuration(false);
-    sslConf.addResource("ssl-server.xml");
-    sslConf.addResource("ssl-client.xml");
+    Configuration sslConf = KeyStoreTestUtil.getSslConfig();
 
     clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
     clientSslFactory.init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
index 0ce835f..1917287 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFactory;
 public class ServerSocketUtil {
 
   private static final Log LOG = LogFactory.getLog(ServerSocketUtil.class);
+  private static Random rand = new Random();
 
   /**
    * Port scan & allocate is how most other apps find ports
@@ -38,13 +39,15 @@ public class ServerSocketUtil {
    * @throws IOException
    */
   public static int getPort(int port, int retries) throws IOException {
-    Random rand = new Random();
     int tryPort = port;
     int tries = 0;
     while (true) {
-      if (tries > 0) {
+      if (tries > 0 || tryPort == 0) {
         tryPort = port + rand.nextInt(65535 - port);
       }
+      if (tryPort == 0) {
+        continue;
+      }
       LOG.info("Using port " + tryPort);
       try (ServerSocket s = new ServerSocket(tryPort)) {
         return tryPort;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
index 07cae8b..453ae48 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
@@ -37,7 +37,6 @@ import java.security.KeyPair;
 import java.security.KeyPairGenerator;
 import java.security.KeyStore;
 import java.security.NoSuchAlgorithmException;
-import java.security.PrivateKey;
 import java.security.SecureRandom;
 import java.security.cert.Certificate;
 import java.security.cert.X509Certificate;
@@ -49,8 +48,6 @@ import java.security.InvalidKeyException;
 import java.security.NoSuchProviderException;
 import java.security.SignatureException;
 import java.security.cert.CertificateEncodingException;
-import java.security.cert.CertificateException;
-import java.security.cert.CertificateFactory;
 import javax.security.auth.x500.X500Principal;
 import org.bouncycastle.x509.X509V1CertificateGenerator;
 
@@ -233,8 +230,8 @@ public class KeyStoreTestUtil {
     String trustKS = null;
     String trustPassword = "trustP";
 
-    File sslClientConfFile = new File(sslConfDir + "/ssl-client.xml");
-    File sslServerConfFile = new File(sslConfDir + "/ssl-server.xml");
+    File sslClientConfFile = new File(sslConfDir, getClientSSLConfigFileName());
+    File sslServerConfFile = new File(sslConfDir, getServerSSLConfigFileName());
 
     Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
 
@@ -312,8 +309,44 @@ public class KeyStoreTestUtil {
   }
 
   /**
+   * Returns the client SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @return client SSL configuration file name
+   */
+  public static String getClientSSLConfigFileName() {
+    return getSSLConfigFileName("ssl-client");
+  }
+
+  /**
+   * Returns the server SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @return client SSL configuration file name
+   */
+  public static String getServerSSLConfigFileName() {
+    return getSSLConfigFileName("ssl-server");
+  }
+
+  /**
+   * Returns an SSL configuration file name.  Under parallel test
+   * execution, this file name is parameterized by a unique ID to ensure that
+   * concurrent tests don't collide on an SSL configuration file.
+   *
+   * @param base the base of the file name
+   * @return SSL configuration file name for base
+   */
+  private static String getSSLConfigFileName(String base) {
+    String testUniqueForkId = System.getProperty("test.unique.fork.id");
+    String fileSuffix = testUniqueForkId != null ? "-" + testUniqueForkId : "";
+    return base + fileSuffix + ".xml";
+  }
+
+  /**
    * Creates SSL configuration.
-   * 
+   *
    * @param mode SSLFactory.Mode mode to configure
    * @param keystore String keystore file
    * @param password String store password, or null to avoid setting store
@@ -410,4 +443,19 @@ public class KeyStoreTestUtil {
       throw e;
     }
   }
+
+  /**
+   * Get the SSL configuration
+   * @return {@link Configuration} instance with ssl configs loaded
+   */
+  public static Configuration getSslConfig(){
+    Configuration sslConf = new Configuration(false);
+    String sslServerConfFile = KeyStoreTestUtil.getServerSSLConfigFileName();
+    String sslClientConfFile = KeyStoreTestUtil.getClientSSLConfigFileName();
+    sslConf.addResource(sslServerConfFile);
+    sslConf.addResource(sslClientConfFile);
+    sslConf.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile);
+    sslConf.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile);
+    return sslConf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f0f984e4/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 636e063..293c279 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -90,7 +90,7 @@
     <enforced.maven.version>[3.0.2,)</enforced.maven.version>
 
     <!-- Plugin versions and config -->
-    <maven-surefire-plugin.argLine>-Xmx4096m -XX:MaxPermSize=768m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
+    <maven-surefire-plugin.argLine>-Xmx2048m -XX:MaxPermSize=768m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
     <maven-surefire-plugin.version>2.17</maven-surefire-plugin.version>
     <maven-surefire-report-plugin.version>${maven-surefire-plugin.version}</maven-surefire-report-plugin.version>
     <maven-failsafe-plugin.version>${maven-surefire-plugin.version}</maven-failsafe-plugin.version>