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 ar...@apache.org on 2016/03/14 21:00:10 UTC

[01/20] hadoop git commit: HADOOP-11404. Clarify the "expected client Kerberos principal is null" authorization message. Contributed by Stephen Chu

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-1312 4f98af414 -> 11045091b


HADOOP-11404. Clarify the "expected client Kerberos principal is null" authorization message. Contributed by Stephen Chu


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

Branch: refs/heads/HDFS-1312
Commit: 318c9b68b059981796f2742b4b7ee604ccdc47e5
Parents: 2e040d3
Author: Harsh J <ha...@cloudera.com>
Authored: Thu Mar 10 17:05:09 2016 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Thu Mar 10 17:05:09 2016 +0530

----------------------------------------------------------------------
 .../security/authorize/ServiceAuthorizationManager.java | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/318c9b68/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
index 5d29516..9da95dc 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
@@ -116,11 +116,13 @@ public class ServiceAuthorizationManager {
     }
     if((clientPrincipal != null && !clientPrincipal.equals(user.getUserName())) || 
        acls.length != 2  || !acls[0].isUserAllowed(user) || acls[1].isUserAllowed(user)) {
-      AUDITLOG.warn(AUTHZ_FAILED_FOR + user + " for protocol=" + protocol
-          + ", expected client Kerberos principal is " + clientPrincipal);
-      throw new AuthorizationException("User " + user + 
-          " is not authorized for protocol " + protocol + 
-          ", expected client Kerberos principal is " + clientPrincipal);
+      String cause = clientPrincipal != null ?
+          ": this service is only accessible by " + clientPrincipal :
+          ": denied by configured ACL";
+      AUDITLOG.warn(AUTHZ_FAILED_FOR + user
+          + " for protocol=" + protocol + cause);
+      throw new AuthorizationException("User " + user +
+          " is not authorized for protocol " + protocol + cause);
     }
     if (addr != null) {
       String hostAddress = addr.getHostAddress();


[10/20] hadoop git commit: HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)

Posted by ar...@apache.org.
HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)


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

Branch: refs/heads/HDFS-1312
Commit: 682adc6ba9db3bed94fd4ea3d83761db6abfe695
Parents: e01c6ea
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Fri Mar 11 15:03:40 2016 +0900
Committer: Masatake Iwasaki <iw...@apache.org>
Committed: Fri Mar 11 15:03:40 2016 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 33 ++++++----
 .../src/main/resources/core-default.xml         |  9 ++-
 .../java/org/apache/hadoop/ipc/TestRPC.java     | 68 ++++++++++++++++++++
 3 files changed, 91 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 8d87957..3ae1d67 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -386,7 +386,7 @@ public class Client {
     private Socket socket = null;                 // connected socket
     private DataInputStream in;
     private DataOutputStream out;
-    private int rpcTimeout;
+    private final int rpcTimeout;
     private int maxIdleTime; //connections will be culled if it was idle for 
     //maxIdleTime msecs
     private final RetryPolicy connectionRetryPolicy;
@@ -394,8 +394,9 @@ public class Client {
     private int maxRetriesOnSocketTimeouts;
     private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
     private final boolean tcpLowLatency; // if T then use low-delay QoS
-    private boolean doPing; //do we need to send ping message
-    private int pingInterval; // how often sends ping to the server in msecs
+    private final boolean doPing; //do we need to send ping message
+    private final int pingInterval; // how often sends ping to the server
+    private final int soTimeout; // used by ipc ping and rpc timeout
     private ByteArrayOutputStream pingRequest; // ping message
     
     // currently active calls
@@ -434,6 +435,9 @@ public class Client {
         pingHeader.writeDelimitedTo(pingRequest);
       }
       this.pingInterval = remoteId.getPingInterval();
+      this.soTimeout =
+          (rpcTimeout == 0 || (doPing && pingInterval < rpcTimeout))?
+              this.pingInterval : this.rpcTimeout;
       this.serviceClass = serviceClass;
       if (LOG.isDebugEnabled()) {
         LOG.debug("The ping interval is " + this.pingInterval + " ms.");
@@ -484,12 +488,12 @@ public class Client {
 
       /* Process timeout exception
        * if the connection is not going to be closed or 
-       * is not configured to have a RPC timeout, send a ping.
-       * (if rpcTimeout is not set to be 0, then RPC should timeout.
-       * otherwise, throw the timeout exception.
+       * the RPC is not timed out yet, send a ping.
        */
-      private void handleTimeout(SocketTimeoutException e) throws IOException {
-        if (shouldCloseConnection.get() || !running.get() || rpcTimeout > 0) {
+      private void handleTimeout(SocketTimeoutException e, int waiting)
+          throws IOException {
+        if (shouldCloseConnection.get() || !running.get() ||
+            (0 < rpcTimeout && rpcTimeout <= waiting)) {
           throw e;
         } else {
           sendPing();
@@ -503,11 +507,13 @@ public class Client {
        */
       @Override
       public int read() throws IOException {
+        int waiting = 0;
         do {
           try {
             return super.read();
           } catch (SocketTimeoutException e) {
-            handleTimeout(e);
+            waiting += soTimeout;
+            handleTimeout(e, waiting);
           }
         } while (true);
       }
@@ -520,11 +526,13 @@ public class Client {
        */
       @Override
       public int read(byte[] buf, int off, int len) throws IOException {
+        int waiting = 0;
         do {
           try {
             return super.read(buf, off, len);
           } catch (SocketTimeoutException e) {
-            handleTimeout(e);
+            waiting += soTimeout;
+            handleTimeout(e, waiting);
           }
         } while (true);
       }
@@ -632,10 +640,7 @@ public class Client {
           }
           
           NetUtils.connect(this.socket, server, connectionTimeout);
-          if (rpcTimeout > 0) {
-            pingInterval = rpcTimeout;  // rpcTimeout overwrites pingInterval
-          }
-          this.socket.setSoTimeout(pingInterval);
+          this.socket.setSoTimeout(soTimeout);
           return;
         } catch (ConnectTimeoutException toe) {
           /* Check for an address change and update the local reference.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 187f923..5037113 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1054,7 +1054,7 @@
   <value>true</value>
   <description>Send a ping to the server when timeout on reading the response,
   if set to true. If no failure is detected, the client retries until at least
-  a byte is read.
+  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
   </description>
 </property>
 
@@ -1071,10 +1071,9 @@
   <name>ipc.client.rpc-timeout.ms</name>
   <value>0</value>
   <description>Timeout on waiting response from server, in milliseconds.
-  Currently this timeout works only when ipc.client.ping is set to true
-  because it uses the same facilities with IPC ping.
-  The timeout overrides the ipc.ping.interval and client will throw exception
-  instead of sending ping when the interval is passed.
+  If ipc.client.ping is set to true and this rpc-timeout is greater than
+  the value of ipc.ping.interval, the effective value of the rpc-timeout is
+  rounded up to multiple of ipc.ping.interval.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/682adc6b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index 99bfc61..929b82b 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -1043,6 +1043,74 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
+  /**
+   *  Test RPC timeout when ipc.client.ping is false.
+   */
+  @Test(timeout=30000)
+  public void testClientRpcTimeoutWithoutPing() throws Exception {
+    final Server server = new RPC.Builder(conf)
+        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
+        .setBindAddress(ADDRESS).setPort(0)
+        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
+        .build();
+    server.start();
+
+    final Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
+    final TestProtocol proxy =
+        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+
+    try {
+      proxy.sleep(3000);
+      fail("RPC should time out.");
+    } catch (SocketTimeoutException e) {
+      LOG.info("got expected timeout.", e);
+    } finally {
+      server.stop();
+      RPC.stopProxy(proxy);
+    }
+  }
+
+  /**
+   *  Test RPC timeout greater than ipc.ping.interval.
+   */
+  @Test(timeout=30000)
+  public void testClientRpcTimeoutGreaterThanPingInterval() throws Exception {
+    final Server server = new RPC.Builder(conf)
+        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
+        .setBindAddress(ADDRESS).setPort(0)
+        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
+        .build();
+    server.start();
+
+    final Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
+    conf.setInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 800);
+    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
+    final TestProtocol proxy =
+        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+
+    // should not time out.
+    proxy.sleep(300);
+
+    // should not time out because effective rpc-timeout is
+    // multiple of ping interval: 1600 (= 800 * (1000 / 800 + 1))
+    proxy.sleep(1300);
+
+    try {
+      proxy.sleep(2000);
+      fail("RPC should time out.");
+    } catch (SocketTimeoutException e) {
+      LOG.info("got expected timeout.", e);
+    } finally {
+      server.stop();
+      RPC.stopProxy(proxy);
+    }
+  }
+
   public static void main(String[] args) throws Exception {
     new TestRPC().testCallsInternal(conf);
   }


[02/20] hadoop git commit: YARN-4696. Improving EntityGroupFSTimelineStore on exception handling, test setup, and concurrency. (Steve Loughran via gtcarrera9)

Posted by ar...@apache.org.
YARN-4696. Improving EntityGroupFSTimelineStore on exception handling, test setup, and concurrency. (Steve Loughran via gtcarrera9)


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

Branch: refs/heads/HDFS-1312
Commit: d49cfb350454c2dfa2f3eb70f79b6d5030ce7bec
Parents: 318c9b6
Author: Li Lu <gt...@apache.org>
Authored: Thu Mar 10 10:51:55 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Thu Mar 10 10:51:55 2016 -0800

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../hadoop/yarn/client/api/TimelineClient.java  |   4 +-
 .../api/impl/FileSystemTimelineWriter.java      |  51 ++---
 .../client/api/impl/TimelineClientImpl.java     |  13 ++
 .../yarn/client/api/impl/TimelineWriter.java    |  40 +++-
 .../timeline/webapp/TimelineWebServices.java    |  12 +-
 .../yarn/server/timeline/EntityCacheItem.java   |  40 ++--
 .../timeline/EntityGroupFSTimelineStore.java    | 204 ++++++++++++++-----
 .../hadoop/yarn/server/timeline/LogInfo.java    |  11 +-
 .../TestEntityGroupFSTimelineStore.java         |   8 +-
 10 files changed, 279 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 61d1d72..ff4b493 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -1747,6 +1747,12 @@ public class YarnConfiguration extends Configuration {
   public static final long
       TIMELINE_SERVICE_CLIENT_INTERNAL_TIMERS_TTL_SECS_DEFAULT = 7 * 60;
 
+  // This is temporary solution. The configuration will be deleted once we have
+  // the FileSystem API to check whether append operation is supported or not.
+  public static final String TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND
+      = TIMELINE_SERVICE_PREFIX
+      + "entity-file.fs-support-append";
+
   // mark app-history related configs @Private as application history is going
   // to be integrated into the timeline service
   @Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
index 258b9f5..09298b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.client.api;
 
+import java.io.Flushable;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -41,7 +42,8 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
  */
 @Public
 @Evolving
-public abstract class TimelineClient extends AbstractService {
+public abstract class TimelineClient extends AbstractService implements
+    Flushable {
 
   /**
    * Create a timeline client. The current UGI when the user initialize the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
index aa1f1f8..9e719b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.Closeable;
+import java.io.FileNotFoundException;
 import java.io.Flushable;
 import java.io.IOException;
 import java.net.URI;
@@ -78,12 +79,6 @@ public class FileSystemTimelineWriter extends TimelineWriter{
   private static final Log LOG = LogFactory
       .getLog(FileSystemTimelineWriter.class);
 
-  // This is temporary solution. The configuration will be deleted once we have
-  // the FileSystem API to check whether append operation is supported or not.
-  private static final String TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND
-      = YarnConfiguration.TIMELINE_SERVICE_PREFIX
-          + "entity-file.fs-support-append";
-
   // App log directory must be readable by group so server can access logs
   // and writable by group so it can be deleted by server
   private static final short APP_LOG_DIR_PERMISSIONS = 0770;
@@ -122,20 +117,10 @@ public class FileSystemTimelineWriter extends TimelineWriter{
           .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
       YarnConfiguration
           .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR_DEFAULT));
+    fs = FileSystem.newInstance(activePath.toUri(), fsConf);
 
-    String scheme = activePath.toUri().getScheme();
-    if (scheme == null) {
-      scheme = FileSystem.getDefaultUri(fsConf).getScheme();
-    }
-    if (scheme != null) {
-      String disableCacheName = String.format("fs.%s.impl.disable.cache",
-          scheme);
-      fsConf.setBoolean(disableCacheName, true);
-    }
-
-    fs = activePath.getFileSystem(fsConf);
     if (!fs.exists(activePath)) {
-      throw new IOException(activePath + " does not exist");
+      throw new FileNotFoundException(activePath + " does not exist");
     }
 
     summaryEntityTypes = new HashSet<String>(
@@ -168,7 +153,8 @@ public class FileSystemTimelineWriter extends TimelineWriter{
             timerTaskTTL);
 
     this.isAppendSupported =
-        conf.getBoolean(TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND, true);
+        conf.getBoolean(
+            YarnConfiguration.TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND, true);
 
     objMapper = createObjectMapper();
 
@@ -181,7 +167,7 @@ public class FileSystemTimelineWriter extends TimelineWriter{
               + "=" + cleanIntervalSecs + ", " +
           YarnConfiguration.TIMELINE_SERVICE_CLIENT_FD_RETAIN_SECS
               + "=" + ttl + ", " +
-          TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYFILE_FS_SUPPORT_APPEND
               + "=" + isAppendSupported + ", " +
           YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR
               + "=" + activePath);
@@ -196,6 +182,11 @@ public class FileSystemTimelineWriter extends TimelineWriter{
   }
 
   @Override
+  public String toString() {
+    return "FileSystemTimelineWriter writing to " + activePath;
+  }
+
+  @Override
   public TimelinePutResponse putEntities(
       ApplicationAttemptId appAttemptId, TimelineEntityGroupId groupId,
       TimelineEntity... entities) throws IOException, YarnException {
@@ -263,9 +254,20 @@ public class FileSystemTimelineWriter extends TimelineWriter{
   }
 
   @Override
-  public void close() throws Exception {
-    if (this.logFDsCache != null) {
-      this.logFDsCache.close();
+  public synchronized void close() throws Exception {
+    if (logFDsCache != null) {
+      LOG.debug("Closing cache");
+      logFDsCache.flush();
+      logFDsCache.close();
+      logFDsCache = null;
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (logFDsCache != null) {
+      LOG.debug("Flushing cache");
+      logFDsCache.flush();
     }
   }
 
@@ -333,6 +335,9 @@ public class FileSystemTimelineWriter extends TimelineWriter{
       if (writerClosed()) {
         prepareForWrite();
       }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Writing entity list of size " + entities.size());
+      }
       for (TimelineEntity entity : entities) {
         getObjectMapper().writeValue(getJsonGenerator(), entity);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index 195a661..ef46229 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -326,6 +326,13 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   @Override
+  public void flush() throws IOException {
+    if (timelineWriter != null) {
+      timelineWriter.flush();
+    }
+  }
+
+  @Override
   public TimelinePutResponse putEntities(
       TimelineEntity... entities) throws IOException, YarnException {
     return timelineWriter.putEntities(entities);
@@ -432,6 +439,12 @@ public class TimelineClientImpl extends TimelineClient {
     operateDelegationToken(cancelDTAction);
   }
 
+  @Override
+  public String toString() {
+    return super.toString() + " with timeline server " + resURI
+        + " and writer " + timelineWriter;
+  }
+
   private Object operateDelegationToken(
       final PrivilegedExceptionAction<?> action)
       throws IOException, YarnException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
index c616e63..9590f4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineWriter.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.client.api.impl;
 
+import java.io.Flushable;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
@@ -48,7 +50,7 @@ import com.sun.jersey.api.client.WebResource;
  */
 @Private
 @Unstable
-public abstract class TimelineWriter {
+public abstract class TimelineWriter implements Flushable {
 
   private static final Log LOG = LogFactory
       .getLog(TimelineWriter.class);
@@ -68,6 +70,16 @@ public abstract class TimelineWriter {
     // DO NOTHING
   }
 
+  @Override
+  public void flush() throws IOException {
+    // DO NOTHING
+  }
+
+  @Override
+  public String toString() {
+    return "Timeline writer posting to " + resURI;
+  }
+
   public TimelinePutResponse putEntities(
       TimelineEntity... entities) throws IOException, YarnException {
     TimelineEntities entitiesContainer = new TimelineEntities();
@@ -104,19 +116,27 @@ public abstract class TimelineWriter {
         }
       });
     } catch (UndeclaredThrowableException e) {
-      throw new IOException(e.getCause());
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException)cause;
+      } else {
+        throw new IOException(cause);
+      }
     } catch (InterruptedException ie) {
-      throw new IOException(ie);
+      throw (IOException)new InterruptedIOException().initCause(ie);
     }
     if (resp == null ||
         resp.getClientResponseStatus() != ClientResponse.Status.OK) {
       String msg =
           "Failed to get the response from the timeline server.";
       LOG.error(msg);
-      if (LOG.isDebugEnabled() && resp != null) {
-        String output = resp.getEntity(String.class);
-        LOG.debug("HTTP error code: " + resp.getStatus()
-            + " Server response : \n" + output);
+      if (resp != null) {
+        msg += " HTTP error code: " + resp.getStatus();
+        if (LOG.isDebugEnabled()) {
+          String output = resp.getEntity(String.class);
+          LOG.debug("HTTP error code: " + resp.getStatus()
+              + " Server response : \n" + output);
+        }
       }
       throw new YarnException(msg);
     }
@@ -128,10 +148,16 @@ public abstract class TimelineWriter {
   public ClientResponse doPostingObject(Object object, String path) {
     WebResource webResource = client.resource(resURI);
     if (path == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("POST to " + resURI);
+      }
       return webResource.accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .post(ClientResponse.class, object);
     } else if (path.equals("domain")) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("PUT to " + resURI +"/" + path);
+      }
       return webResource.path(path).accept(MediaType.APPLICATION_JSON)
           .type(MediaType.APPLICATION_JSON)
           .put(ClientResponse.class, object);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
index eb47ef2..e1e684b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/webapp/TimelineWebServices.java
@@ -129,9 +129,9 @@ public class TimelineWebServices {
           getUser(req));
     } catch (NumberFormatException e) {
       throw new BadRequestException(
-          "windowStart, windowEnd or limit is not a numeric value.");
+        "windowStart, windowEnd, fromTs or limit is not a numeric value: " + e);
     } catch (IllegalArgumentException e) {
-      throw new BadRequestException("requested invalid field.");
+      throw new BadRequestException("requested invalid field: " + e);
     } catch (Exception e) {
       LOG.error("Error getting entities", e);
       throw new WebApplicationException(e,
@@ -160,8 +160,7 @@ public class TimelineWebServices {
           parseFieldsStr(fields, ","),
           getUser(req));
     } catch (IllegalArgumentException e) {
-      throw new BadRequestException(
-          "requested invalid field.");
+      throw new BadRequestException(e);
     } catch (Exception e) {
       LOG.error("Error getting entity", e);
       throw new WebApplicationException(e,
@@ -201,8 +200,9 @@ public class TimelineWebServices {
           parseLongStr(limit),
           getUser(req));
     } catch (NumberFormatException e) {
-      throw new BadRequestException(
-          "windowStart, windowEnd or limit is not a numeric value.");
+      throw (BadRequestException)new BadRequestException(
+          "windowStart, windowEnd or limit is not a numeric value.")
+          .initCause(e);
     } catch (Exception e) {
       LOG.error("Error getting entity timelines", e);
       throw new WebApplicationException(e,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
index efbf994..7eec7c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityCacheItem.java
@@ -107,30 +107,30 @@ public class EntityCacheItem {
           store.init(config);
           store.start();
         }
-        TimelineDataManager tdm = new TimelineDataManager(store,
-            aclManager);
-        tdm.init(config);
-        tdm.start();
-        List<LogInfo> removeList = new ArrayList<LogInfo>();
-        for (LogInfo log : appLogs.getDetailLogs()) {
-          LOG.debug("Try refresh logs for {}", log.getFilename());
-          // Only refresh the log that matches the cache id
-          if (log.matchesGroupId(groupId)) {
-            Path appDirPath = appLogs.getAppDirPath();
-            if (fs.exists(log.getPath(appDirPath))) {
-              LOG.debug("Refresh logs for cache id {}", groupId);
-              log.parseForStore(tdm, appDirPath, appLogs.isDone(), jsonFactory,
-                  objMapper, fs);
-            } else {
-              // The log may have been removed, remove the log
-              removeList.add(log);
-              LOG.info("File {} no longer exists, remove it from log list",
-                  log.getPath(appDirPath));
+        List<LogInfo> removeList = new ArrayList<>();
+        try(TimelineDataManager tdm =
+                new TimelineDataManager(store, aclManager)) {
+          tdm.init(config);
+          tdm.start();
+          for (LogInfo log : appLogs.getDetailLogs()) {
+            LOG.debug("Try refresh logs for {}", log.getFilename());
+            // Only refresh the log that matches the cache id
+            if (log.matchesGroupId(groupId)) {
+              Path appDirPath = appLogs.getAppDirPath();
+              if (fs.exists(log.getPath(appDirPath))) {
+                LOG.debug("Refresh logs for cache id {}", groupId);
+                log.parseForStore(tdm, appDirPath, appLogs.isDone(),
+                    jsonFactory, objMapper, fs);
+              } else {
+                // The log may have been removed, remove the log
+                removeList.add(log);
+                LOG.info("File {} no longer exists, removing it from log list",
+                    log.getPath(appDirPath));
+              }
             }
           }
         }
         appLogs.getDetailLogs().removeAll(removeList);
-        tdm.close();
       }
       updateRefreshTimeToNow();
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
index b1fbd13..34a2072 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/EntityGroupFSTimelineStore.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -55,6 +56,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -71,12 +73,13 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Plugin timeline storage to support timeline server v1.5 API. This storage
  * uses a file system to store timeline entities in their groups.
  */
-public class EntityGroupFSTimelineStore extends AbstractService
+public class EntityGroupFSTimelineStore extends CompositeService
     implements TimelineStore {
 
   static final String DOMAIN_LOG_PREFIX = "domainlog-";
@@ -110,6 +113,7 @@ public class EntityGroupFSTimelineStore extends AbstractService
   private ConcurrentMap<ApplicationId, AppLogs> appIdLogMap =
       new ConcurrentHashMap<ApplicationId, AppLogs>();
   private ScheduledThreadPoolExecutor executor;
+  private AtomicBoolean stopExecutors = new AtomicBoolean(false);
   private FileSystem fs;
   private ObjectMapper objMapper;
   private JsonFactory jsonFactory;
@@ -128,7 +132,8 @@ public class EntityGroupFSTimelineStore extends AbstractService
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     summaryStore = createSummaryStore();
-    summaryStore.init(conf);
+    addService(summaryStore);
+
     long logRetainSecs = conf.getLong(
         YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RETAIN_SECONDS,
         YarnConfiguration
@@ -170,17 +175,28 @@ public class EntityGroupFSTimelineStore extends AbstractService
       });
     cacheIdPlugins = loadPlugIns(conf);
     // Initialize yarn client for application status
-    yarnClient = YarnClient.createYarnClient();
-    yarnClient.init(conf);
+    yarnClient = createAndInitYarnClient(conf);
+    // if non-null, hook its lifecycle up
+    addIfService(yarnClient);
+    activeRootPath = new Path(conf.get(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
+        YarnConfiguration
+            .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR_DEFAULT));
+    doneRootPath = new Path(conf.get(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR,
+        YarnConfiguration
+            .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT));
+    fs = activeRootPath.getFileSystem(conf);
     super.serviceInit(conf);
   }
 
   private List<TimelineEntityGroupPlugin> loadPlugIns(Configuration conf)
       throws RuntimeException {
-    Collection<String> pluginNames = conf.getStringCollection(
+    Collection<String> pluginNames = conf.getTrimmedStringCollection(
         YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES);
     List<TimelineEntityGroupPlugin> pluginList
         = new LinkedList<TimelineEntityGroupPlugin>();
+    Exception caught = null;
     for (final String name : pluginNames) {
       LOG.debug("Trying to load plugin class {}", name);
       TimelineEntityGroupPlugin cacheIdPlugin = null;
@@ -191,10 +207,11 @@ public class EntityGroupFSTimelineStore extends AbstractService
                 clazz, conf);
       } catch (Exception e) {
         LOG.warn("Error loading plugin " + name, e);
+        caught = e;
       }
 
       if (cacheIdPlugin == null) {
-        throw new RuntimeException("No class defined for " + name);
+        throw new RuntimeException("No class defined for " + name, caught);
       }
       LOG.info("Load plugin class {}", cacheIdPlugin.getClass().getName());
       pluginList.add(cacheIdPlugin);
@@ -210,8 +227,9 @@ public class EntityGroupFSTimelineStore extends AbstractService
 
   @Override
   protected void serviceStart() throws Exception {
+
+    super.serviceStart();
     LOG.info("Starting {}", getName());
-    yarnClient.start();
     summaryStore.start();
 
     Configuration conf = getConfig();
@@ -219,16 +237,10 @@ public class EntityGroupFSTimelineStore extends AbstractService
     aclManager.setTimelineStore(summaryStore);
     summaryTdm = new TimelineDataManager(summaryStore, aclManager);
     summaryTdm.init(conf);
-    summaryTdm.start();
-    activeRootPath = new Path(conf.get(
-        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
-        YarnConfiguration
-            .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR_DEFAULT));
-    doneRootPath = new Path(conf.get(
-        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR,
-        YarnConfiguration
-            .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT));
-    fs = activeRootPath.getFileSystem(conf);
+    addService(summaryTdm);
+    // start child services that aren't already started
+    super.serviceStart();
+
     if (!fs.exists(activeRootPath)) {
       fs.mkdirs(activeRootPath);
       fs.setPermission(activeRootPath, ACTIVE_DIR_PERMISSION);
@@ -257,7 +269,8 @@ public class EntityGroupFSTimelineStore extends AbstractService
         YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_THREADS,
         YarnConfiguration
             .TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_THREADS_DEFAULT);
-    LOG.info("Scanning active directory every {} seconds", scanIntervalSecs);
+    LOG.info("Scanning active directory {} every {} seconds", activeRootPath,
+        scanIntervalSecs);
     LOG.info("Cleaning logs every {} seconds", cleanerIntervalSecs);
 
     executor = new ScheduledThreadPoolExecutor(numThreads,
@@ -267,12 +280,12 @@ public class EntityGroupFSTimelineStore extends AbstractService
         TimeUnit.SECONDS);
     executor.scheduleAtFixedRate(new EntityLogCleaner(), cleanerIntervalSecs,
         cleanerIntervalSecs, TimeUnit.SECONDS);
-    super.serviceStart();
   }
 
   @Override
   protected void serviceStop() throws Exception {
     LOG.info("Stopping {}", getName());
+    stopExecutors.set(true);
     if (executor != null) {
       executor.shutdown();
       if (executor.isTerminating()) {
@@ -286,18 +299,9 @@ public class EntityGroupFSTimelineStore extends AbstractService
         }
       }
     }
-    if (summaryTdm != null) {
-      summaryTdm.stop();
-    }
-    if (summaryStore != null) {
-      summaryStore.stop();
-    }
-    if (yarnClient != null) {
-      yarnClient.stop();
-    }
     synchronized (cachedLogs) {
       for (EntityCacheItem cacheItem : cachedLogs.values()) {
-        cacheItem.getStore().close();
+        ServiceOperations.stopQuietly(cacheItem.getStore());
       }
     }
     super.serviceStop();
@@ -305,17 +309,34 @@ public class EntityGroupFSTimelineStore extends AbstractService
 
   @InterfaceAudience.Private
   @VisibleForTesting
-  void scanActiveLogs() throws IOException {
-    RemoteIterator<FileStatus> iter = fs.listStatusIterator(activeRootPath);
+  int scanActiveLogs() throws IOException {
+    RemoteIterator<FileStatus> iter = list(activeRootPath);
+    int logsToScanCount = 0;
     while (iter.hasNext()) {
       FileStatus stat = iter.next();
-      ApplicationId appId = parseApplicationId(stat.getPath().getName());
+      String name = stat.getPath().getName();
+      ApplicationId appId = parseApplicationId(name);
       if (appId != null) {
         LOG.debug("scan logs for {} in {}", appId, stat.getPath());
+        logsToScanCount++;
         AppLogs logs = getAndSetActiveLog(appId, stat.getPath());
         executor.execute(new ActiveLogParser(logs));
+      } else {
+        LOG.debug("Unable to parse entry {}", name);
       }
     }
+    return logsToScanCount;
+  }
+
+  /**
+   * List a directory, returning an iterator which will fail fast if this
+   * service has been stopped
+   * @param path path to list
+   * @return an iterator over the contents of the directory
+   * @throws IOException
+   */
+  private RemoteIterator<FileStatus> list(Path path) throws IOException {
+    return new StoppableRemoteIterator(fs.listStatusIterator(path));
   }
 
   private AppLogs createAndPutAppLogsIfAbsent(ApplicationId appId,
@@ -377,11 +398,11 @@ public class EntityGroupFSTimelineStore extends AbstractService
    */
   @InterfaceAudience.Private
   @VisibleForTesting
-  static void cleanLogs(Path dirpath, FileSystem fs, long retainMillis)
+  void cleanLogs(Path dirpath, FileSystem fs, long retainMillis)
       throws IOException {
     long now = Time.now();
     // Depth first search from root directory for all application log dirs
-    RemoteIterator<FileStatus> iter = fs.listStatusIterator(dirpath);
+    RemoteIterator<FileStatus> iter = list(dirpath);
     while (iter.hasNext()) {
       FileStatus stat = iter.next();
       if (stat.isDirectory()) {
@@ -456,7 +477,42 @@ public class EntityGroupFSTimelineStore extends AbstractService
             bucket1, bucket2, appId.toString()));
   }
 
-  // This method has to be synchronized to control traffic to RM
+  /**
+   * Create and initialize the YARN Client. Tests may override/mock this.
+   * If they return null, then {@link #getAppState(ApplicationId)} MUST
+   * also be overridden
+   * @param conf configuration
+   * @return the yarn client, or null.
+   *
+   */
+  @VisibleForTesting
+  protected YarnClient createAndInitYarnClient(Configuration conf) {
+    YarnClient client = YarnClient.createYarnClient();
+    client.init(conf);
+    return client;
+  }
+
+  /**
+   * Get the application state.
+   * @param appId application ID
+   * @return the state or {@link AppState#UNKNOWN} if it could not
+   * be determined
+   * @throws IOException on IO problems
+   */
+  @VisibleForTesting
+  protected AppState getAppState(ApplicationId appId) throws IOException {
+    return getAppState(appId, yarnClient);
+  }
+
+  /**
+   * Ask the RM for the state of the application.
+   * This method has to be synchronized to control traffic to RM
+   * @param appId application ID
+   * @param yarnClient
+   * @return the state or {@link AppState#UNKNOWN} if it could not
+   * be determined
+   * @throws IOException
+   */
   private static synchronized AppState getAppState(ApplicationId appId,
       YarnClient yarnClient) throws IOException {
     AppState appState = AppState.ACTIVE;
@@ -474,9 +530,12 @@ public class EntityGroupFSTimelineStore extends AbstractService
     return appState;
   }
 
+  /**
+   * Application states,
+   */
   @InterfaceAudience.Private
   @VisibleForTesting
-  enum AppState {
+  public enum AppState {
     ACTIVE,
     UNKNOWN,
     COMPLETED
@@ -526,7 +585,7 @@ public class EntityGroupFSTimelineStore extends AbstractService
       if (!isDone()) {
         LOG.debug("Try to parse summary log for log {} in {}",
             appId, appDirPath);
-        appState = EntityGroupFSTimelineStore.getAppState(appId, yarnClient);
+        appState = getAppState(appId);
         long recentLogModTime = scanForLogs();
         if (appState == AppState.UNKNOWN) {
           if (Time.now() - recentLogModTime > unknownActiveMillis) {
@@ -559,8 +618,7 @@ public class EntityGroupFSTimelineStore extends AbstractService
     long scanForLogs() throws IOException {
       LOG.debug("scanForLogs on {}", appDirPath);
       long newestModTime = 0;
-      RemoteIterator<FileStatus> iterAttempt =
-          fs.listStatusIterator(appDirPath);
+      RemoteIterator<FileStatus> iterAttempt = list(appDirPath);
       while (iterAttempt.hasNext()) {
         FileStatus statAttempt = iterAttempt.next();
         LOG.debug("scanForLogs on {}", statAttempt.getPath().getName());
@@ -572,8 +630,7 @@ public class EntityGroupFSTimelineStore extends AbstractService
           continue;
         }
         String attemptDirName = statAttempt.getPath().getName();
-        RemoteIterator<FileStatus> iterCache
-            = fs.listStatusIterator(statAttempt.getPath());
+        RemoteIterator<FileStatus> iterCache = list(statAttempt.getPath());
         while (iterCache.hasNext()) {
           FileStatus statCache = iterCache.next();
           if (!statCache.isFile()) {
@@ -659,14 +716,34 @@ public class EntityGroupFSTimelineStore extends AbstractService
     }
   }
 
+  /**
+   * Extract any nested throwable forwarded from IPC operations.
+   * @param e exception
+   * @return either the exception passed an an argument, or any nested
+   * exception which was wrapped inside an {@link UndeclaredThrowableException}
+   */
+  private Throwable extract(Exception e) {
+    Throwable t = e;
+    if (e instanceof UndeclaredThrowableException && e.getCause() != null) {
+      t = e.getCause();
+    }
+    return t;
+  }
+
   private class EntityLogScanner implements Runnable {
     @Override
     public void run() {
       LOG.debug("Active scan starting");
       try {
-        scanActiveLogs();
+        int scanned = scanActiveLogs();
+        LOG.debug("Scanned {} active applications", scanned);
       } catch (Exception e) {
-        LOG.error("Error scanning active files", e);
+        Throwable t = extract(e);
+        if (t instanceof InterruptedException) {
+          LOG.info("File scanner interrupted");
+        } else {
+          LOG.error("Error scanning active files", t);
+        }
       }
       LOG.debug("Active scan complete");
     }
@@ -690,7 +767,12 @@ public class EntityGroupFSTimelineStore extends AbstractService
         }
         LOG.debug("End parsing summary logs. ");
       } catch (Exception e) {
-        LOG.error("Error processing logs for " + appLogs.getAppId(), e);
+        Throwable t = extract(e);
+        if (t instanceof InterruptedException) {
+          LOG.info("Log parser interrupted");
+        } else {
+          LOG.error("Error processing logs for " + appLogs.getAppId(), t);
+        }
       }
     }
   }
@@ -702,7 +784,12 @@ public class EntityGroupFSTimelineStore extends AbstractService
       try {
         cleanLogs(doneRootPath, fs, logRetainMillis);
       } catch (Exception e) {
-        LOG.error("Error cleaning files", e);
+        Throwable t = extract(e);
+        if (t instanceof InterruptedException) {
+          LOG.info("Cleaner interrupted");
+        } else {
+          LOG.error("Error cleaning files", e);
+        }
       }
       LOG.debug("Cleaner finished");
     }
@@ -892,4 +979,29 @@ public class EntityGroupFSTimelineStore extends AbstractService
   public void put(TimelineDomain domain) throws IOException {
     summaryStore.put(domain);
   }
+
+  /**
+   * This is a special remote iterator whose {@link #hasNext()} method
+   * returns false if {@link #stopExecutors} is true.
+   *
+   * This provides an implicit shutdown of all iterative file list and scan
+   * operations without needing to implement it in the while loops themselves.
+   */
+  private class StoppableRemoteIterator implements RemoteIterator<FileStatus> {
+    private final RemoteIterator<FileStatus> remote;
+
+    public StoppableRemoteIterator(RemoteIterator<FileStatus> remote) {
+      this.remote = remote;
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      return !stopExecutors.get() && remote.hasNext();
+    }
+
+    @Override
+    public FileStatus next() throws IOException {
+      return remote.next();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java
index 4caed8d..bc80175 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/main/java/org/apache/hadoop/yarn/server/timeline/LogInfo.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.timeline;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -103,7 +104,8 @@ abstract class LogInfo {
     LOG.debug("Parsing for log dir {} on attempt {}", appDirPath,
         attemptDirName);
     Path logPath = getPath(appDirPath);
-    if (fs.exists(logPath)) {
+    FileStatus status = fs.getFileStatus(logPath);
+    if (status != null) {
       long startTime = Time.monotonicNow();
       try {
         LOG.debug("Parsing {} at offset {}", logPath, offset);
@@ -112,8 +114,11 @@ abstract class LogInfo {
         LOG.info("Parsed {} entities from {} in {} msec",
             count, logPath, Time.monotonicNow() - startTime);
       } catch (RuntimeException e) {
-        if (e.getCause() instanceof JsonParseException) {
-          // If AppLogs cannot parse this log, it may be corrupted
+        // If AppLogs cannot parse this log, it may be corrupted or just empty
+        if (e.getCause() instanceof JsonParseException &&
+            (status.getLen() > 0 || offset > 0)) {
+          // log on parse problems if the file as been read in the past or
+          // is visibly non-empty
           LOG.info("Log {} appears to be corrupted. Skip. ", logPath);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d49cfb35/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
index e43b705..3e5bc06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestEntityGroupFSTimelineStore.java
@@ -116,14 +116,14 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
           EntityGroupPlugInForTest.class.getName());
     }
     store.init(config);
-    store.start();
     store.setFs(fs);
+    store.start();
   }
 
   @After
   public void tearDown() throws Exception {
-    fs.delete(TEST_APP_DIR_PATH, true);
     store.stop();
+    fs.delete(TEST_APP_DIR_PATH, true);
   }
 
   @AfterClass
@@ -222,7 +222,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     fs.mkdirs(dirPathEmpty);
 
     // Should retain all logs after this run
-    EntityGroupFSTimelineStore.cleanLogs(TEST_DONE_DIR_PATH, fs, 10000);
+    store.cleanLogs(TEST_DONE_DIR_PATH, fs, 10000);
     assertTrue(fs.exists(irrelevantDirPath));
     assertTrue(fs.exists(irrelevantFilePath));
     assertTrue(fs.exists(filePath));
@@ -239,7 +239,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
     // Touch the third application by creating a new dir
     fs.mkdirs(new Path(dirPathHold, "holdByMe"));
 
-    EntityGroupFSTimelineStore.cleanLogs(TEST_DONE_DIR_PATH, fs, 1000);
+    store.cleanLogs(TEST_DONE_DIR_PATH, fs, 1000);
 
     // Verification after the second cleaner call
     assertTrue(fs.exists(irrelevantDirPath));


[19/20] hadoop git commit: HDFS-9941. Do not log StandbyException on NN, other minor logging fixes. Contributed by Arpit Agarwal.

Posted by ar...@apache.org.
HDFS-9941. Do not log StandbyException on NN, other minor logging fixes. Contributed by Arpit Agarwal.


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

Branch: refs/heads/HDFS-1312
Commit: 5644137adad30c84e40d2c4719627b3aabc73628
Parents: f291d82
Author: Chris Nauroth <cn...@apache.org>
Authored: Mon Mar 14 09:54:54 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Mar 14 09:54:54 2016 -0700

----------------------------------------------------------------------
 .../BlockUnderConstructionFeature.java          | 45 ++++++++++++--------
 .../blockmanagement/DecommissionManager.java    |  8 +++-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 18 +++++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  4 ++
 4 files changed, 55 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
index b46b470..1a93033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
@@ -33,11 +33,13 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCSt
  */
 public class BlockUnderConstructionFeature {
   private BlockUCState blockUCState;
+  private static final ReplicaUnderConstruction[] NO_REPLICAS =
+      new ReplicaUnderConstruction[0];
 
   /**
    * Block replicas as assigned when the block was allocated.
    */
-  private ReplicaUnderConstruction[] replicas;
+  private ReplicaUnderConstruction[] replicas = NO_REPLICAS;
 
   /**
    * Index of the primary data node doing the recovery. Useful for log
@@ -120,7 +122,7 @@ public class BlockUnderConstructionFeature {
   }
 
   public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.length;
+    return replicas.length;
   }
 
   /**
@@ -130,7 +132,7 @@ public class BlockUnderConstructionFeature {
    */
   void updateStorageScheduledSize(BlockInfoStriped storedBlock) {
     assert storedBlock.getUnderConstructionFeature() == this;
-    if (replicas == null) {
+    if (replicas.length == 0) {
       return;
     }
     final int dataBlockNum = storedBlock.getDataBlockNum();
@@ -182,12 +184,10 @@ public class BlockUnderConstructionFeature {
 
   List<ReplicaUnderConstruction> getStaleReplicas(long genStamp) {
     List<ReplicaUnderConstruction> staleReplicas = new ArrayList<>();
-    if (replicas != null) {
-      // Remove replicas with wrong gen stamp. The replica list is unchanged.
-      for (ReplicaUnderConstruction r : replicas) {
-        if (genStamp != r.getGenerationStamp()) {
-          staleReplicas.add(r);
-        }
+    // Remove replicas with wrong gen stamp. The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        staleReplicas.add(r);
       }
     }
     return staleReplicas;
@@ -201,7 +201,7 @@ public class BlockUnderConstructionFeature {
   public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
-    if (replicas == null || replicas.length == 0) {
+    if (replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
           " BlockUnderConstructionFeature.initializeBlockRecovery:" +
           " No blocks found, lease removed.");
@@ -252,7 +252,7 @@ public class BlockUnderConstructionFeature {
   /** Add the reported replica if it is not already in the replica list. */
   void addReplicaIfNotPresent(DatanodeStorageInfo storage,
       Block reportedBlock, ReplicaState rState) {
-    if (replicas == null) {
+    if (replicas.length == 0) {
       replicas = new ReplicaUnderConstruction[1];
       replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage,
           rState);
@@ -295,15 +295,24 @@ public class BlockUnderConstructionFeature {
       .append(", truncateBlock=").append(truncateBlock)
       .append(", primaryNodeIndex=").append(primaryNodeIndex)
       .append(", replicas=[");
-    if (replicas != null) {
-      int i = 0;
-      for (ReplicaUnderConstruction r : replicas) {
-        r.appendStringTo(sb);
-        if (++i < replicas.length) {
-          sb.append(", ");
-        }
+    int i = 0;
+    for (ReplicaUnderConstruction r : replicas) {
+      r.appendStringTo(sb);
+      if (++i < replicas.length) {
+        sb.append(", ");
       }
     }
     sb.append("]}");
   }
+  
+  public void appendUCPartsConcise(StringBuilder sb) {
+    sb.append("replicas=");
+    int i = 0;
+    for (ReplicaUnderConstruction r : replicas) {
+      sb.append(r.getExpectedStorageLocation().getDatanodeDescriptor());
+      if (++i < replicas.length) {
+        sb.append(", ");
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 2a5d63c..480670a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -280,6 +281,10 @@ public class DecommissionManager {
       BlockCollection bc,
       DatanodeDescriptor srcNode, NumberReplicas num,
       Iterable<DatanodeStorageInfo> storages) {
+    if (!NameNode.blockStateChangeLog.isInfoEnabled()) {
+      return;
+    }
+
     int curReplicas = num.liveReplicas();
     int curExpectedReplicas = blockManager.getExpectedReplicaNum(block);
     StringBuilder nodeList = new StringBuilder();
@@ -288,7 +293,8 @@ public class DecommissionManager {
       nodeList.append(node);
       nodeList.append(" ");
     }
-    LOG.info("Block: " + block + ", Expected Replicas: "
+    NameNode.blockStateChangeLog.info(
+        "Block: " + block + ", Expected Replicas: "
         + curExpectedReplicas + ", live replicas: " + curReplicas
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", decommissioned replicas: " + num.decommissioned()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index cc08528..41fd869 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -848,10 +848,26 @@ class FSDirWriteFileOp {
     assert fsn.hasWriteLock();
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
         isStriped);
-    NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
+    logAllocatedBlock(src, b);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
 
+  private static void logAllocatedBlock(String src, BlockInfo b) {
+    if (!NameNode.stateChangeLog.isInfoEnabled()) {
+      return;
+    }
+    StringBuilder sb = new StringBuilder(150);
+    sb.append("BLOCK* allocate ");
+    b.appendStringTo(sb);
+    sb.append(", ");
+    BlockUnderConstructionFeature uc = b.getUnderConstructionFeature();
+    if (uc != null) {
+      uc.appendUCPartsConcise(sb);
+    }
+    sb.append(" for " + src);
+    NameNode.stateChangeLog.info(sb.toString());
+  }
+
   private static void setNewINodeStoragePolicy(BlockManager bm, INodeFile
       inode, INodesInPath iip, boolean isLazyPersist)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5644137a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 6dff1bc..eb47580 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -163,6 +163,7 @@ import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.ipc.RefreshRegistry;
 import org.apache.hadoop.ipc.RefreshResponse;
@@ -494,6 +495,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
         FSLimitException.PathComponentTooLongException.class,
         FSLimitException.MaxDirectoryItemsExceededException.class,
         UnresolvedPathException.class);
+
+    clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class);
+
     clientRpcServer.setTracer(nn.tracer);
     if (serviceRpcServer != null) {
       serviceRpcServer.setTracer(nn.tracer);


[14/20] hadoop git commit: Revert "HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)"

Posted by ar...@apache.org.
Revert "HADOOP-12672. RPC timeout should not override IPC ping interval (iwasakims)"

This reverts commit 682adc6ba9db3bed94fd4ea3d83761db6abfe695.


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

Branch: refs/heads/HDFS-1312
Commit: 754299695b778b9b602e46836c35a3ac9474d7f8
Parents: 247a790
Author: Steve Loughran <st...@apache.org>
Authored: Fri Mar 11 17:00:17 2016 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Mar 11 17:00:17 2016 +0000

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ipc/Client.java | 33 ++++------
 .../src/main/resources/core-default.xml         |  9 +--
 .../java/org/apache/hadoop/ipc/TestRPC.java     | 68 --------------------
 3 files changed, 19 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
index 3ae1d67..8d87957 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
@@ -386,7 +386,7 @@ public class Client {
     private Socket socket = null;                 // connected socket
     private DataInputStream in;
     private DataOutputStream out;
-    private final int rpcTimeout;
+    private int rpcTimeout;
     private int maxIdleTime; //connections will be culled if it was idle for 
     //maxIdleTime msecs
     private final RetryPolicy connectionRetryPolicy;
@@ -394,9 +394,8 @@ public class Client {
     private int maxRetriesOnSocketTimeouts;
     private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
     private final boolean tcpLowLatency; // if T then use low-delay QoS
-    private final boolean doPing; //do we need to send ping message
-    private final int pingInterval; // how often sends ping to the server
-    private final int soTimeout; // used by ipc ping and rpc timeout
+    private boolean doPing; //do we need to send ping message
+    private int pingInterval; // how often sends ping to the server in msecs
     private ByteArrayOutputStream pingRequest; // ping message
     
     // currently active calls
@@ -435,9 +434,6 @@ public class Client {
         pingHeader.writeDelimitedTo(pingRequest);
       }
       this.pingInterval = remoteId.getPingInterval();
-      this.soTimeout =
-          (rpcTimeout == 0 || (doPing && pingInterval < rpcTimeout))?
-              this.pingInterval : this.rpcTimeout;
       this.serviceClass = serviceClass;
       if (LOG.isDebugEnabled()) {
         LOG.debug("The ping interval is " + this.pingInterval + " ms.");
@@ -488,12 +484,12 @@ public class Client {
 
       /* Process timeout exception
        * if the connection is not going to be closed or 
-       * the RPC is not timed out yet, send a ping.
+       * is not configured to have a RPC timeout, send a ping.
+       * (if rpcTimeout is not set to be 0, then RPC should timeout.
+       * otherwise, throw the timeout exception.
        */
-      private void handleTimeout(SocketTimeoutException e, int waiting)
-          throws IOException {
-        if (shouldCloseConnection.get() || !running.get() ||
-            (0 < rpcTimeout && rpcTimeout <= waiting)) {
+      private void handleTimeout(SocketTimeoutException e) throws IOException {
+        if (shouldCloseConnection.get() || !running.get() || rpcTimeout > 0) {
           throw e;
         } else {
           sendPing();
@@ -507,13 +503,11 @@ public class Client {
        */
       @Override
       public int read() throws IOException {
-        int waiting = 0;
         do {
           try {
             return super.read();
           } catch (SocketTimeoutException e) {
-            waiting += soTimeout;
-            handleTimeout(e, waiting);
+            handleTimeout(e);
           }
         } while (true);
       }
@@ -526,13 +520,11 @@ public class Client {
        */
       @Override
       public int read(byte[] buf, int off, int len) throws IOException {
-        int waiting = 0;
         do {
           try {
             return super.read(buf, off, len);
           } catch (SocketTimeoutException e) {
-            waiting += soTimeout;
-            handleTimeout(e, waiting);
+            handleTimeout(e);
           }
         } while (true);
       }
@@ -640,7 +632,10 @@ public class Client {
           }
           
           NetUtils.connect(this.socket, server, connectionTimeout);
-          this.socket.setSoTimeout(soTimeout);
+          if (rpcTimeout > 0) {
+            pingInterval = rpcTimeout;  // rpcTimeout overwrites pingInterval
+          }
+          this.socket.setSoTimeout(pingInterval);
           return;
         } catch (ConnectTimeoutException toe) {
           /* Check for an address change and update the local reference.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 5037113..187f923 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -1054,7 +1054,7 @@
   <value>true</value>
   <description>Send a ping to the server when timeout on reading the response,
   if set to true. If no failure is detected, the client retries until at least
-  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
+  a byte is read.
   </description>
 </property>
 
@@ -1071,9 +1071,10 @@
   <name>ipc.client.rpc-timeout.ms</name>
   <value>0</value>
   <description>Timeout on waiting response from server, in milliseconds.
-  If ipc.client.ping is set to true and this rpc-timeout is greater than
-  the value of ipc.ping.interval, the effective value of the rpc-timeout is
-  rounded up to multiple of ipc.ping.interval.
+  Currently this timeout works only when ipc.client.ping is set to true
+  because it uses the same facilities with IPC ping.
+  The timeout overrides the ipc.ping.interval and client will throw exception
+  instead of sending ping when the interval is passed.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75429969/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index 929b82b..99bfc61 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -1043,74 +1043,6 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
-  /**
-   *  Test RPC timeout when ipc.client.ping is false.
-   */
-  @Test(timeout=30000)
-  public void testClientRpcTimeoutWithoutPing() throws Exception {
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0)
-        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
-        .build();
-    server.start();
-
-    final Configuration conf = new Configuration();
-    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
-    final TestProtocol proxy =
-        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-
-    try {
-      proxy.sleep(3000);
-      fail("RPC should time out.");
-    } catch (SocketTimeoutException e) {
-      LOG.info("got expected timeout.", e);
-    } finally {
-      server.stop();
-      RPC.stopProxy(proxy);
-    }
-  }
-
-  /**
-   *  Test RPC timeout greater than ipc.ping.interval.
-   */
-  @Test(timeout=30000)
-  public void testClientRpcTimeoutGreaterThanPingInterval() throws Exception {
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0)
-        .setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
-        .build();
-    server.start();
-
-    final Configuration conf = new Configuration();
-    conf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    conf.setInt(CommonConfigurationKeys.IPC_PING_INTERVAL_KEY, 800);
-    conf.setInt(CommonConfigurationKeys.IPC_CLIENT_RPC_TIMEOUT_KEY, 1000);
-    final TestProtocol proxy =
-        RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-
-    // should not time out.
-    proxy.sleep(300);
-
-    // should not time out because effective rpc-timeout is
-    // multiple of ping interval: 1600 (= 800 * (1000 / 800 + 1))
-    proxy.sleep(1300);
-
-    try {
-      proxy.sleep(2000);
-      fail("RPC should time out.");
-    } catch (SocketTimeoutException e) {
-      LOG.info("got expected timeout.", e);
-    } finally {
-      server.stop();
-      RPC.stopProxy(proxy);
-    }
-  }
-
   public static void main(String[] args) throws Exception {
     new TestRPC().testCallsInternal(conf);
   }


[08/20] hadoop git commit: HADOOP-12899. External distribution stitching scripts do not work correctly on Windows. Contributed by Chris Nauroth.

Posted by ar...@apache.org.
HADOOP-12899. External distribution stitching scripts do not work correctly on Windows. 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/adf1cdf3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/adf1cdf3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/adf1cdf3

Branch: refs/heads/HDFS-1312
Commit: adf1cdf3d5abed0dae76a4967129ce64f2e16c2f
Parents: 79961ec
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Mar 10 14:49:08 2016 -0800
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Mar 10 14:49:08 2016 -0800

----------------------------------------------------------------------
 hadoop-dist/pom.xml | 24 +++++++++++++-----------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adf1cdf3/hadoop-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index ce1bd92..42e74cb 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -90,16 +90,17 @@
                 <id>dist</id>
                 <phase>prepare-package</phase>
                 <goals>
-                    <goal>exec</goal>
+                  <goal>exec</goal>
                 </goals>
                 <configuration>
-                    <executable>${basedir}/../dev-support/bin/dist-layout-stitching</executable>
-                    <workingDirectory>${project.build.directory}</workingDirectory>
-                    <requiresOnline>false</requiresOnline>
-                    <arguments>
-                       <argument>${project.version}</argument>
-                       <argument>${project.build.directory}</argument>
-                    </arguments>
+                  <executable>${shell-executable}</executable>
+                  <workingDirectory>${project.build.directory}</workingDirectory>
+                  <requiresOnline>false</requiresOnline>
+                  <arguments>
+                    <argument>${basedir}/../dev-support/bin/dist-layout-stitching</argument>
+                    <argument>${project.version}</argument>
+                    <argument>${project.build.directory}</argument>
+                  </arguments>
                 </configuration>
               </execution>
               <execution>
@@ -109,12 +110,13 @@
                   <goal>exec</goal>
                 </goals>
                 <configuration>
-                    <executable>${basedir}/../dev-support/bin/dist-tar-stitching</executable>
+                    <executable>${shell-executable}</executable>
                     <workingDirectory>${project.build.directory}</workingDirectory>
                     <requiresOnline>false</requiresOnline>
                     <arguments>
-                       <argument>${project.version}</argument>
-                       <argument>${project.build.directory}</argument>
+                      <argument>${basedir}/../dev-support/bin/dist-tar-stitching</argument>
+                      <argument>${project.version}</argument>
+                      <argument>${project.build.directory}</argument>
                     </arguments>
                 </configuration>
               </execution>


[11/20] hadoop git commit: MAPREDUCE-6520. Migrate MR Client test cases part 1.

Posted by ar...@apache.org.
MAPREDUCE-6520. Migrate MR Client test cases part 1.


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

Branch: refs/heads/HDFS-1312
Commit: ef68b441957c6deb65adae2272d60ac929d00733
Parents: 682adc6
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:20:55 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:20:55 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 +++++++++++----
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 ++++-----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++++++--
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 ++
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 ++++++++++-----
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 ++
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +++++++++-
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 +++++----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 ++++++++++++-
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +++++++++-
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 ++++-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 ++-
 .../hadoop/mapreduce/TestTaskContext.java       |  2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 +++++
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 +++
 .../lib/chain/TestSingleElementChain.java       |  4 ++
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++++++++++------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 ++-
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +++-
 .../lib/map/TestMultithreadedMapper.java        | 12 +++++-
 .../lib/output/TestJobOutputCommitter.java      | 16 ++++++--
 .../lib/output/TestMRMultipleOutputs.java       | 17 ++++++++-
 .../TestMRKeyFieldBasedComparator.java          | 20 +++++++---
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +++++-
 25 files changed, 285 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index b47d5ef..e2d75ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,17 +17,30 @@
  */
 package org.apache.hadoop.conf;
 
-import org.junit.Assert;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-
-import java.io.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.Utils;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -40,6 +53,7 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
+  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 3cd0668..277c0fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.mapred;
 
-import junit.framework.TestCase;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.junit.After;
+import org.junit.Before;
 
-import java.io.File;
 import java.io.IOException;
 
 /**
@@ -38,7 +37,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase extends TestCase {
+public abstract class HadoopTestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -140,8 +139,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -164,7 +163,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,7 +181,6 @@ public abstract class HadoopTestCase extends TestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
-    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index d2ea74e..1f657cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,6 +34,13 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -140,17 +147,19 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
+  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 81b53cc..3141235 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,12 +30,16 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bad06e9..bed545e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.mapred;
 
-import java.io.File;
-import java.io.IOException;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -27,9 +24,18 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
-import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -80,12 +86,13 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @Override
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-  
+
+  @Test
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -199,6 +206,7 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -245,23 +253,27 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
+  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -269,6 +281,7 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 8d35dcf..07b1306 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -59,6 +61,7 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
+  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 37cb91f..0933ece 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,12 +21,29 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -67,6 +84,7 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 34a4d2c..35b3f24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index e5c6d75..3a9cb9e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.mapred.lib;
 
-import java.io.IOException;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -30,12 +25,19 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs extends TestCase {
-  
+public class TestMultipleInputs {
+
+  @Test
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -48,6 +50,7 @@ public class TestMultipleInputs extends TestCase {
        .getClass());
   }
 
+  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index 59c0a97..f3e5893 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,7 +24,23 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -32,17 +48,23 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -62,6 +84,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -75,6 +98,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
+  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 7e224cd..1059d29 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,26 +22,45 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index d5afe63..338f117 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,6 +30,13 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -145,7 +152,8 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 5d36c92..7520f38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -68,7 +71,8 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index bf742c4..67daaa4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,6 +33,8 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 2dfcf41..46024bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,6 +29,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -51,6 +55,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -89,6 +94,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -114,6 +120,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -139,6 +146,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -163,6 +171,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index 971ea68..aaaaf51 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,6 +30,11 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -63,6 +68,7 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index 06cfe1c..f78ac70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -42,6 +45,7 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
+  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 37f9364..81a3249 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,25 +18,40 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-import java.sql.*;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-//import org.apache.hadoop.examples.DBCountPageView;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.db.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -109,11 +124,13 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
+  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -170,6 +187,7 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
+  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index c868050..632c40e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -139,7 +141,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -152,7 +154,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index 14c64bd..d86ddd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -120,7 +123,8 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-  
+
+  @Test
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -144,6 +148,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -182,6 +187,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 52b0e70..5096192 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,23 +23,33 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 49b59ca..19b712f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,6 +33,11 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -54,15 +59,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -219,6 +224,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -238,6 +244,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -264,6 +271,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
+  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index 6c432dd..babd20e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,23 +27,36 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -57,6 +70,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -64,6 +78,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
+  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 3a2b831..0d75d2f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import java.io.*;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +30,15 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -45,8 +52,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-  
-  private void testComparator(String keySpec, int expect) 
+
+  private void testComparator(String keySpec, int expect)
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -93,7 +100,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-  
+
+  @Test
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -117,7 +125,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef68b441/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 3492089..391e482 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,6 +27,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -35,7 +41,8 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -76,6 +83,7 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
+  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -104,6 +112,7 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
+  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[04/20] hadoop git commit: YARN-4696. Improving EntityGroupFSTimelineStore on exception handling, test setup, and concurrency.

Posted by ar...@apache.org.
YARN-4696. Improving EntityGroupFSTimelineStore on exception handling, test setup, and concurrency.

This commit amends commit d49cfb350454c2dfa2f3eb70f79b6d5030ce7bec with a missed test file.


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

Branch: refs/heads/HDFS-1312
Commit: 017d2c127b9cbd75d3e31467172ed832f27ef826
Parents: 9a79b73
Author: Li Lu <gt...@apache.org>
Authored: Thu Mar 10 13:02:28 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Thu Mar 10 13:02:28 2016 -0800

----------------------------------------------------------------------
 .../TestOverrideTimelineStoreYarnClient.java    | 56 ++++++++++++++++++++
 1 file changed, 56 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/017d2c12/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestOverrideTimelineStoreYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestOverrideTimelineStoreYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestOverrideTimelineStoreYarnClient.java
new file mode 100644
index 0000000..c190266
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestOverrideTimelineStoreYarnClient.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hadoop.yarn.server.timeline;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestOverrideTimelineStoreYarnClient {
+
+  @Test
+  public void testLifecycleAndOverride() throws Throwable {
+    YarnConfiguration conf = new YarnConfiguration();
+    try(NoRMStore store = new NoRMStore()) {
+      store.init(conf);
+      store.start();
+      Assert.assertEquals(EntityGroupFSTimelineStore.AppState.ACTIVE,
+          store.getAppState(ApplicationId.newInstance(1, 1)));
+      store.stop();
+    }
+  }
+
+  private static class NoRMStore extends EntityGroupFSTimelineStore {
+    @Override
+    protected YarnClient createAndInitYarnClient(Configuration conf) {
+      return null;
+    }
+
+    @Override
+    protected AppState getAppState(ApplicationId appId)
+        throws IOException {
+      return AppState.ACTIVE;
+    }
+  }
+}


[06/20] hadoop git commit: HDFS-9934. ReverseXML oiv processor should bail out if the XML file's layoutVersion doesn't match oiv's (cmccabe)

Posted by ar...@apache.org.
HDFS-9934. ReverseXML oiv processor should bail out if the XML file's layoutVersion doesn't match oiv's (cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: bd49354c6d6387620b0de2219eab1714ec2d64f8
Parents: 500875d
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 10 13:41:06 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 10 13:41:06 2016 -0800

----------------------------------------------------------------------
 .../OfflineImageReconstructor.java              | 11 +++++++
 .../TestOfflineImageViewer.java                 | 34 ++++++++++++++++++++
 2 files changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd49354c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
index e5d0e2c..73f4a22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotDiffSection.DiffEntry;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -1493,6 +1494,16 @@ class OfflineImageReconstructor {
       throw new IOException("The <version> section doesn't contain " +
           "the layoutVersion.");
     }
+    if (layoutVersion.intValue() !=
+        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION) {
+      throw new IOException("Layout version mismatch.  This oiv tool " +
+          "handles layout version " +
+          NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION + ", but the " +
+          "XML file has <layoutVersion> " + layoutVersion + ".  Please " +
+          "either re-generate the XML file with the proper layout version, " +
+          "or manually edit the XML file to be usable with this version " +
+          "of the oiv tool.");
+    }
     fileSummaryBld.setOndiskVersion(onDiskVersion);
     fileSummaryBld.setLayoutVersion(layoutVersion);
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd49354c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
index c7a6ae9..5d5fea6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
@@ -29,6 +29,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintStream;
+import java.io.PrintWriter;
 import java.io.RandomAccessFile;
 import java.io.StringReader;
 import java.net.HttpURLConnection;
@@ -64,6 +65,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -499,4 +501,36 @@ public class TestOfflineImageViewer {
     Assert.assertEquals("",
       GenericTestUtils.getFilesDiff(reverseImageXml, reverseImage2Xml));
   }
+
+  /**
+   * Tests that the ReverseXML processor doesn't accept XML files with the wrong
+   * layoutVersion.
+   */
+  @Test
+  public void testReverseXmlWrongLayoutVersion() throws Throwable {
+    File imageWrongVersion = new File(tempDir, "imageWrongVersion.xml");
+    PrintWriter writer = new PrintWriter(imageWrongVersion, "UTF-8");
+    try {
+      writer.println("<?xml version=\"1.0\"?>");
+      writer.println("<fsimage>");
+      writer.println("<version>");
+      writer.println(String.format("<layoutVersion>%d</layoutVersion>",
+          NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION + 1));
+      writer.println("<onDiskVersion>1</onDiskVersion>");
+      writer.println("<oivRevision>" +
+          "545bbef596c06af1c3c8dca1ce29096a64608478</oivRevision>");
+      writer.println("</version>");
+      writer.println("</fsimage>");
+    } finally {
+      writer.close();
+    }
+    try {
+      OfflineImageReconstructor.run(imageWrongVersion.getAbsolutePath(),
+          imageWrongVersion.getAbsolutePath() + ".out"); 
+      Assert.fail("Expected OfflineImageReconstructor to fail with " +
+          "version mismatch.");
+    } catch (Throwable t) {
+      GenericTestUtils.assertExceptionContains("Layout version mismatch.", t);
+    }
+  }
 }


[18/20] hadoop git commit: YARN-4545. Allow YARN distributed shell to use ATS v1.5 APIs. Li Lu via junping_du

Posted by ar...@apache.org.
YARN-4545. Allow YARN distributed shell to use ATS v1.5 APIs. Li Lu via junping_du


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

Branch: refs/heads/HDFS-1312
Commit: f291d82cd49c04a81380bc45c97c279d791b571c
Parents: 658ee95
Author: Junping Du <ju...@apache.org>
Authored: Mon Mar 14 08:28:38 2016 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Mon Mar 14 08:28:38 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |  13 ++
 .../pom.xml                                     |  21 ++++
 .../distributedshell/ApplicationMaster.java     |  68 +++++++----
 .../DistributedShellTimelinePlugin.java         |  79 ++++++++++++
 .../distributedshell/package-info.java          |  19 +++
 .../distributedshell/TestDistributedShell.java  | 120 ++++++++++++++++---
 .../yarn/util/timeline/TimelineUtils.java       |  35 ++++++
 .../hadoop/yarn/server/MiniYARNCluster.java     |  10 +-
 .../yarn/server/timeline/TimelineVersion.java   |  31 +++++
 .../server/timeline/TimelineVersionWatcher.java |  47 ++++++++
 .../pom.xml                                     |  16 +++
 .../server/timeline/PluginStoreTestUtils.java   |  51 +++++++-
 12 files changed, 469 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index f23b46e..3362c11 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -298,6 +298,19 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+        <type>test-jar</type>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
index 09a56ea..c118603 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
@@ -121,6 +121,27 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timeline-pluginstorage</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 95dbddc..0f82903 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -99,6 +100,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.log4j.LogManager;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -277,6 +279,9 @@ public class ApplicationMaster {
   // Timeline Client
   @VisibleForTesting
   TimelineClient timelineClient;
+  static final String CONTAINER_ENTITY_GROUP_ID = "CONTAINERS";
+  static final String APPID_TIMELINE_FILTER_NAME = "appId";
+  static final String USER_TIMELINE_FILTER_NAME = "user";
 
   private final String linux_bash_command = "bash";
   private final String windows_command = "cmd /c";
@@ -904,7 +909,7 @@ public class ApplicationMaster {
         applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
       }
       if(applicationMaster.timelineClient != null) {
-        ApplicationMaster.publishContainerStartEvent(
+        applicationMaster.publishContainerStartEvent(
             applicationMaster.timelineClient, container,
             applicationMaster.domainId, applicationMaster.appSubmitterUgi);
       }
@@ -1120,15 +1125,17 @@ public class ApplicationMaster {
       org.apache.commons.io.IOUtils.closeQuietly(ds);
     }
   }
-  
-  private static void publishContainerStartEvent(
-      final TimelineClient timelineClient, Container container, String domainId,
-      UserGroupInformation ugi) {
+
+  private void publishContainerStartEvent(
+      final TimelineClient timelineClient, final Container container,
+      String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
+    entity.addPrimaryFilter(APPID_TIMELINE_FILTER_NAME, container.getId()
+        .getApplicationAttemptId().getApplicationId().toString());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_START.toString());
@@ -1137,28 +1144,27 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<TimelinePutResponse>() {
-        @Override
-        public TimelinePutResponse run() throws Exception {
-          return processTimelineResponseErrors(
-              timelineClient.putEntities(entity));
-        }
-      });
-    } catch (Exception e) {
+      processTimelineResponseErrors(
+          putContainerEntity(timelineClient,
+              container.getId().getApplicationAttemptId(),
+              entity));
+    } catch (YarnException | IOException e) {
       LOG.error("Container start event could not be published for "
-          + container.getId().toString(),
-          e instanceof UndeclaredThrowableException ? e.getCause() : e);
+          + container.getId().toString(), e);
     }
   }
 
-  private static void publishContainerEndEvent(
+  private void publishContainerEndEvent(
       final TimelineClient timelineClient, ContainerStatus container,
       String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getContainerId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
+    entity.addPrimaryFilter(APPID_TIMELINE_FILTER_NAME,
+        container.getContainerId().getApplicationAttemptId()
+            .getApplicationId().toString());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_END.toString());
@@ -1166,22 +1172,38 @@ public class ApplicationMaster {
     event.addEventInfo("Exit Status", container.getExitStatus());
     entity.addEvent(event);
     try {
-      TimelinePutResponse response = timelineClient.putEntities(entity);
-      processTimelineResponseErrors(response);
+      processTimelineResponseErrors(
+          putContainerEntity(timelineClient,
+              container.getContainerId().getApplicationAttemptId(),
+              entity));
     } catch (YarnException | IOException e) {
       LOG.error("Container end event could not be published for "
           + container.getContainerId().toString(), e);
     }
   }
 
-  private static void publishApplicationAttemptEvent(
+  private TimelinePutResponse putContainerEntity(
+      TimelineClient timelineClient, ApplicationAttemptId currAttemptId,
+      TimelineEntity entity)
+      throws YarnException, IOException {
+    if (TimelineUtils.timelineServiceV1_5Enabled(conf)) {
+      TimelineEntityGroupId groupId = TimelineEntityGroupId.newInstance(
+          currAttemptId.getApplicationId(),
+          CONTAINER_ENTITY_GROUP_ID);
+      return timelineClient.putEntities(currAttemptId, groupId, entity);
+    } else {
+      return timelineClient.putEntities(entity);
+    }
+  }
+
+  private void publishApplicationAttemptEvent(
       final TimelineClient timelineClient, String appAttemptId,
       DSEvent appEvent, String domainId, UserGroupInformation ugi) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(appAttemptId);
     entity.setEntityType(DSEntity.DS_APP_ATTEMPT.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter(USER_TIMELINE_FILTER_NAME, ugi.getShortUserName());
     TimelineEvent event = new TimelineEvent();
     event.setEventType(appEvent.toString());
     event.setTimestamp(System.currentTimeMillis());
@@ -1197,7 +1219,7 @@ public class ApplicationMaster {
     }
   }
 
-  private static TimelinePutResponse processTimelineResponseErrors(
+  private TimelinePutResponse processTimelineResponseErrors(
       TimelinePutResponse response) {
     List<TimelinePutResponse.TimelinePutError> errors = response.getErrors();
     if (errors.size() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
new file mode 100644
index 0000000..55fbd60
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
@@ -0,0 +1,79 @@
+/**
+ * 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.hadoop.yarn.applications.distributedshell;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.TimelineEntityGroupPlugin;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * Timeline v1.5 reader plugin for YARN distributed shell. It tranlsates an
+ * incoming getEntity request to a set of related timeline entity groups, via
+ * the information provided in the primary filter or entity id field.
+ */
+public class DistributedShellTimelinePlugin extends TimelineEntityGroupPlugin {
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityType,
+      NameValuePair primaryFilter, Collection<NameValuePair> secondaryFilters) {
+    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityType)) {
+      if (primaryFilter == null) {
+        return null;
+      }
+      return toEntityGroupId(primaryFilter.getValue().toString());
+    }
+    return null;
+  }
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityId,
+      String entityType) {
+    if (ApplicationMaster.DSEntity.DS_CONTAINER.toString().equals(entityId)) {
+      ContainerId containerId = ConverterUtils.toContainerId(entityId);
+      ApplicationId appId = containerId.getApplicationAttemptId()
+          .getApplicationId();
+      return toEntityGroupId(appId.toString());
+    }
+    return null;
+  }
+
+  @Override
+  public Set<TimelineEntityGroupId> getTimelineEntityGroupId(String entityType,
+      SortedSet<String> entityIds, Set<String> eventTypes) {
+    // Right now this method is not used by TimelineEntityGroupPlugin
+    return null;
+  }
+
+  private Set<TimelineEntityGroupId> toEntityGroupId(String strAppId) {
+    ApplicationId appId = ConverterUtils.toApplicationId(strAppId);
+    TimelineEntityGroupId groupId = TimelineEntityGroupId.newInstance(
+        appId, ApplicationMaster.CONTAINER_ENTITY_GROUP_ID);
+    Set<TimelineEntityGroupId> result = new HashSet<>();
+    result.add(groupId);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
new file mode 100644
index 0000000..299a286
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
@@ -0,0 +1,19 @@
+/**
+ * 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.hadoop.yarn.applications.distributedshell;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 3197875..6536050 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -36,12 +36,19 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
@@ -50,29 +57,50 @@ import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
+import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
+import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.Timeout;
 
 public class TestDistributedShell {
 
   private static final Log LOG =
       LogFactory.getLog(TestDistributedShell.class);
 
-  protected MiniYARNCluster yarnCluster = null;  
+  protected MiniYARNCluster yarnCluster = null;
+  protected MiniDFSCluster hdfsCluster = null;
+  private FileSystem fs = null;
   protected YarnConfiguration conf = null;
   private static final int NUM_NMS = 1;
+  private static final float DEFAULT_TIMELINE_VERSION = 1.0f;
 
   protected final static String APPMASTER_JAR =
       JarFinder.getJar(ApplicationMaster.class);
 
+  @Rule
+  public TimelineVersionWatcher timelineVersionWatcher
+      = new TimelineVersionWatcher();
+  @Rule
+  public Timeout globalTimeout = new Timeout(90000);
+
   @Before
   public void setup() throws Exception {
-    setupInternal(NUM_NMS);
+    setupInternal(NUM_NMS, timelineVersionWatcher.getTimelineVersion());
   }
 
   protected void setupInternal(int numNodeManager) throws Exception {
+    setupInternal(numNodeManager, DEFAULT_TIMELINE_VERSION);
+  }
+
+  private void setupInternal(int numNodeManager, float timelineVersion)
+      throws Exception {
 
     LOG.info("Starting up YARN cluster");
     
@@ -84,6 +112,26 @@ public class TestDistributedShell {
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
     conf.set("mapreduce.jobhistory.address",
         "0.0.0.0:" + ServerSocketUtil.getPort(10021, 10));
+
+    // ATS version specific settings
+    if (timelineVersion == 1.0f) {
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
+      conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+          CommonConfigurationKeysPublic.FS_DEFAULT_NAME_DEFAULT);
+    } else if (timelineVersion == 1.5f) {
+      if (hdfsCluster == null) {
+        HdfsConfiguration hdfsConfig = new HdfsConfiguration();
+        hdfsCluster = new MiniDFSCluster.Builder(hdfsConfig)
+            .numDataNodes(1).build();
+      }
+      fs = hdfsCluster.getFileSystem();
+      PluginStoreTestUtils.prepareFileSystemForPluginStore(fs);
+      PluginStoreTestUtils.prepareConfiguration(conf, hdfsCluster);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_ENTITY_GROUP_PLUGIN_CLASSES,
+          DistributedShellTimelinePlugin.class.getName());
+    } else {
+      Assert.fail("Wrong timeline version number: " + timelineVersion);
+    }
     
     if (yarnCluster == null) {
       yarnCluster =
@@ -138,6 +186,13 @@ public class TestDistributedShell {
         yarnCluster = null;
       }
     }
+    if (hdfsCluster != null) {
+      try {
+        hdfsCluster.shutdown();
+      } finally {
+        hdfsCluster = null;
+      }
+    }
     FileContext fsContext = FileContext.getLocalFSFileContext();
     fsContext
         .delete(
@@ -146,16 +201,28 @@ public class TestDistributedShell {
             true);
   }
   
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithDomain() throws Exception {
     testDSShell(true);
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithoutDomain() throws Exception {
     testDSShell(false);
   }
 
+  @Test
+  @TimelineVersion(1.5f)
+  public void testDSShellWithoutDomainV1_5() throws Exception {
+    testDSShell(false);
+  }
+
+  @Test
+  @TimelineVersion(1.5f)
+  public void testDSShellWithDomainV1_5() throws Exception {
+    testDSShell(true);
+  }
+
   public void testDSShell(boolean haveDomain) throws Exception {
     String[] args = {
         "--jar",
@@ -239,6 +306,24 @@ public class TestDistributedShell {
     LOG.info("Client run completed. Result=" + result);
     Assert.assertTrue(result.get());
 
+    if (timelineVersionWatcher.getTimelineVersion() == 1.5f) {
+      long scanInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS,
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS_DEFAULT
+      );
+      Path doneDir = new Path(
+          YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT
+      );
+      // Wait till the data is moved to done dir, or timeout and fail
+      while (true) {
+        RemoteIterator<FileStatus> iterApps = fs.listStatusIterator(doneDir);
+        if (iterApps.hasNext()) {
+          break;
+        }
+        Thread.sleep(scanInterval * 2);
+      }
+    }
+
     TimelineDomain domain = null;
     if (haveDomain) {
       domain = yarnCluster.getApplicationHistoryServer()
@@ -265,11 +350,18 @@ public class TestDistributedShell {
       Assert.assertEquals("DEFAULT",
           entitiesAttempts.getEntities().get(0).getDomainId());
     }
+    String currAttemptEntityId
+        = entitiesAttempts.getEntities().get(0).getEntityId();
+    ApplicationAttemptId attemptId
+        = ConverterUtils.toApplicationAttemptId(currAttemptEntityId);
+    NameValuePair primaryFilter = new NameValuePair(
+        ApplicationMaster.APPID_TIMELINE_FILTER_NAME,
+        attemptId.getApplicationId().toString());
     TimelineEntities entities = yarnCluster
         .getApplicationHistoryServer()
         .getTimelineStore()
         .getEntities(ApplicationMaster.DSEntity.DS_CONTAINER.toString(), null,
-            null, null, null, null, null, null, null, null);
+            null, null, null, null, primaryFilter, null, null, null);
     Assert.assertNotNull(entities);
     Assert.assertEquals(2, entities.getEntities().size());
     Assert.assertEquals(entities.getEntities().get(0).getEntityType()
@@ -341,7 +433,7 @@ public class TestDistributedShell {
 
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSRestartWithPreviousRunningContainers() throws Exception {
     String[] args = {
         "--jar",
@@ -376,7 +468,7 @@ public class TestDistributedShell {
    * how many attempt failures for previous 2.5 seconds.
    * The application is expected to be successful.
    */
-  @Test(timeout=90000)
+  @Test
   public void testDSAttemptFailuresValidityIntervalSucess() throws Exception {
     String[] args = {
         "--jar",
@@ -414,7 +506,7 @@ public class TestDistributedShell {
    * how many attempt failure for previous 15 seconds.
    * The application is expected to be fail.
    */
-  @Test(timeout=90000)
+  @Test
   public void testDSAttemptFailuresValidityIntervalFailed() throws Exception {
     String[] args = {
         "--jar",
@@ -446,7 +538,7 @@ public class TestDistributedShell {
       Assert.assertFalse(result);
     }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithCustomLogPropertyFile() throws Exception {
     final File basedir =
         new File("target", TestDistributedShell.class.getName());
@@ -541,7 +633,7 @@ public class TestDistributedShell {
     verifyContainerLog(2, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithMultipleArgs() throws Exception {
     String[] args = {
         "--jar",
@@ -575,7 +667,7 @@ public class TestDistributedShell {
     verifyContainerLog(4, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithShellScript() throws Exception {
     final File basedir =
         new File("target", TestDistributedShell.class.getName());
@@ -623,7 +715,7 @@ public class TestDistributedShell {
     verifyContainerLog(1, expectedContent, false, "");
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDSShellWithInvalidArgs() throws Exception {
     Client client = new Client(new Configuration(yarnCluster.getConfig()));
 
@@ -785,7 +877,7 @@ public class TestDistributedShell {
     }
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testContainerLaunchFailureHandling() throws Exception {
     String[] args = {
       "--jar",
@@ -813,7 +905,7 @@ public class TestDistributedShell {
 
   }
 
-  @Test(timeout=90000)
+  @Test
   public void testDebugFlag() throws Exception {
     String[] args = {
         "--jar",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
index 4f838e6..a794e97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
@@ -86,6 +86,41 @@ public class TimelineUtils {
     }
   }
 
+  /**
+   * Returns whether the timeline service is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service is enabled.
+   */
+  public static boolean timelineServiceEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+  }
+
+  /**
+   * Returns the timeline service version. It does not check whether the
+   * timeline service itself is enabled.
+   *
+   * @param conf the configuration
+   * @return the timeline service version as a float.
+   */
+  public static float getTimelineServiceVersion(Configuration conf) {
+    return conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+  }
+
+  /**
+   * Returns whether the timeline service v.1.5 is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service v.1.5 is enabled. V.1.5 refers to a
+   * version equal to 1.5.
+   */
+  public static boolean timelineServiceV1_5Enabled(Configuration conf) {
+    return timelineServiceEnabled(conf) &&
+        Math.abs(getTimelineServiceVersion(conf) - 1.5) < 0.00001;
+  }
+
   public static TimelineAbout createTimelineAbout(String about) {
     TimelineAbout tsInfo = new TimelineAbout(about);
     tsInfo.setHadoopBuildVersion(VersionInfo.getBuildVersion());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 630b7ef..024adc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
@@ -76,6 +75,7 @@ import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.MemoryTimelineStateStore;
 import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -749,8 +749,12 @@ public class MiniYARNCluster extends CompositeService {
       appHistoryServer = new ApplicationHistoryServer();
       conf.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
           MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
-      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
-          MemoryTimelineStore.class, TimelineStore.class);
+      // Only set memory timeline store if timeline v1.5 is not enabled.
+      // Otherwise, caller has the freedom to choose storage impl.
+      if (!TimelineUtils.timelineServiceV1_5Enabled(conf)) {
+        conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
+            MemoryTimelineStore.class, TimelineStore.class);
+      }
       conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,
           MemoryTimelineStateStore.class, TimelineStateStore.class);
       if (!useFixedPorts) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.java
new file mode 100644
index 0000000..57439de
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersion.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.hadoop.yarn.server.timeline;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(value = RetentionPolicy.RUNTIME)
+@Target(value = {ElementType.METHOD})
+public @interface TimelineVersion {
+  float value() default TimelineVersionWatcher.DEFAULT_TIMELINE_VERSION;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java
new file mode 100644
index 0000000..b00f13a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineVersionWatcher.java
@@ -0,0 +1,47 @@
+/**
+ * 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.hadoop.yarn.server.timeline;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TimelineVersionWatcher extends TestWatcher {
+  static final float DEFAULT_TIMELINE_VERSION = 1.0f;
+  private TimelineVersion version;
+
+  @Override
+  protected void starting(Description description) {
+    version = description.getAnnotation(TimelineVersion.class);
+  }
+
+  /**
+   * @return the version number of timeline server for the current test (using
+   * timeline server v1.0 by default)
+   */
+  public float getTimelineVersion() {
+    if(version == null) {
+      return DEFAULT_TIMELINE_VERSION;
+    }
+    return version.value();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
index 71f76d3..1dd301a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/pom.xml
@@ -137,4 +137,20 @@
       <artifactId>jackson-databind</artifactId>
     </dependency>
   </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f291d82c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
index f529b59..c2c4101 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/PluginStoreTestUtils.java
@@ -18,13 +18,16 @@
 package org.apache.hadoop.yarn.server.timeline;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
 import org.codehaus.jackson.JsonFactory;
@@ -48,7 +51,53 @@ import java.util.Set;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
-class PluginStoreTestUtils {
+/**
+ * Utility methods related to the ATS v1.5 plugin storage tests.
+ */
+public class PluginStoreTestUtils {
+
+  /**
+   * For a given file system, setup directories ready to test the plugin storage.
+   *
+   * @param fs a {@link FileSystem} object that the plugin storage will work with
+   * @return the dfsCluster ready to start plugin storage tests.
+   * @throws IOException
+   */
+  public static FileSystem prepareFileSystemForPluginStore(FileSystem fs)
+      throws IOException {
+    Path activeDir = new Path(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR_DEFAULT
+    );
+    Path doneDir = new Path(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT
+    );
+
+    fs.mkdirs(activeDir);
+    fs.mkdirs(doneDir);
+    return fs;
+  }
+
+  /**
+   * Prepare configuration for plugin tests. This method will also add the mini
+   * DFS cluster's info to the configuration.
+   * Note: the test program needs to setup the reader plugin by itself.
+   *
+   * @param conf
+   * @param dfsCluster
+   * @return the modified configuration
+   */
+  public static YarnConfiguration prepareConfiguration(YarnConfiguration conf,
+      MiniDFSCluster dfsCluster) {
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+        dfsCluster.getURI().toString());
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f);
+    conf.setLong(
+        YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SCAN_INTERVAL_SECONDS,
+        1);
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_STORE,
+        EntityGroupFSTimelineStore.class.getName());
+    return conf;
+  }
 
   static FSDataOutputStream createLogFile(Path logPath, FileSystem fs)
       throws IOException {


[12/20] hadoop git commit: Revert "MAPREDUCE-6520. Migrate MR Client test cases part 1."

Posted by ar...@apache.org.
Revert "MAPREDUCE-6520. Migrate MR Client test cases part 1."

This reverts commit ef68b441957c6deb65adae2272d60ac929d00733.


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

Branch: refs/heads/HDFS-1312
Commit: 6876b9f9f990553565eb11aa17635c3874864d7f
Parents: ef68b44
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:50:57 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:50:57 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 ++++-----------
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 +++++----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++------
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 --
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 +++++----------
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 --
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +---------
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 -
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 ++++-----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 +------------
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +---------
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 +----
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 +--
 .../hadoop/mapreduce/TestTaskContext.java       |  2 -
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 -----
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 ---
 .../lib/chain/TestSingleElementChain.java       |  4 --
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++--------------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 +--
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +---
 .../lib/map/TestMultithreadedMapper.java        | 12 +-----
 .../lib/output/TestJobOutputCommitter.java      | 16 ++------
 .../lib/output/TestMRMultipleOutputs.java       | 17 +--------
 .../TestMRKeyFieldBasedComparator.java          | 20 +++-------
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +-----
 25 files changed, 68 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index e2d75ab..b47d5ef 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,30 +17,17 @@
  */
 package org.apache.hadoop.conf;
 
-import org.apache.hadoop.fs.FileUtil;
+import org.junit.Assert;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.apache.hadoop.mapred.Utils;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+
+import java.io.*;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -53,7 +40,6 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
-  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 277c0fd..3cd0668 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.mapred;
 
-import org.apache.hadoop.fs.FileSystem;
+import junit.framework.TestCase;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRConfig;
-import org.junit.After;
-import org.junit.Before;
 
+import java.io.File;
 import java.io.IOException;
 
 /**
@@ -37,7 +38,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase {
+public abstract class HadoopTestCase extends TestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -139,8 +140,8 @@ public abstract class HadoopTestCase {
    *
    * @throws Exception
    */
-  @Before
-  public void setUp() throws Exception {
+  protected void setUp() throws Exception {
+    super.setUp();
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -163,8 +164,7 @@ public abstract class HadoopTestCase {
    *
    * @throws Exception
    */
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,6 +181,7 @@ public abstract class HadoopTestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
+    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index 1f657cf..d2ea74e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,13 +34,6 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import org.junit.Before;
-import org.junit.After;
-import org.junit.Test;
-
-
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -147,19 +140,17 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-  @Before
-  public void setUp() throws Exception {
+
+  protected void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
-  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 3141235..81b53cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,16 +30,12 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bed545e..bad06e9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.mapred;
 
+import java.io.File;
+import java.io.IOException;
+
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -24,18 +27,9 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
+import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
-import org.junit.After;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -86,13 +80,12 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @After
+  @Override
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-
-  @Test
+  
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -206,7 +199,6 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -253,27 +245,23 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
-  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
-  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-
-  @Test
+  
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-
-  @Test
+  
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -281,7 +269,6 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
-  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 07b1306..8d35dcf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -61,7 +59,6 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
-  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 0933ece..37cb91f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,29 +21,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.junit.Test;
+import org.apache.hadoop.mapred.*;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -84,7 +67,6 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 35b3f24..34a4d2c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index 3a9cb9e..e5c6d75 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.mapred.lib;
 
+import java.io.IOException;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -25,19 +30,12 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs {
-
-  @Test
+public class TestMultipleInputs extends TestCase {
+  
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -50,7 +48,6 @@ public class TestMultipleInputs {
        .getClass());
   }
 
-  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index f3e5893..59c0a97 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,23 +24,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.SequenceFileOutputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.hadoop.mapred.*;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -48,23 +32,17 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
-  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -84,7 +62,6 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
-  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -98,7 +75,6 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
-  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 1059d29..7e224cd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,45 +22,26 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
-import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
-  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
-
-  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index 338f117..d5afe63 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,13 +30,6 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -152,8 +145,7 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-
-  @Test
+  
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 7520f38..5d36c92 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,9 +30,6 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -71,8 +68,7 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-
-  @Test
+  
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index 67daaa4..bf742c4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,8 +33,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 46024bc..2dfcf41 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,10 +29,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -55,7 +51,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -94,7 +89,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -120,7 +114,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -146,7 +139,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -171,7 +163,6 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index aaaaf51..971ea68 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,11 +30,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -68,7 +63,6 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index f78ac70..06cfe1c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,9 +26,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -45,7 +42,6 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
-  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 81a3249..37f9364 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,40 +18,25 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
+import java.sql.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+//import org.apache.hadoop.examples.DBCountPageView;
+import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.db.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -124,13 +109,11 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
-  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
-  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -187,7 +170,6 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
-  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index 632c40e..c868050 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,8 +38,6 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -141,7 +139,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @Test
+  @SuppressWarnings("unchecked")
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -154,7 +152,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @Test
+  @SuppressWarnings("unchecked")
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index d86ddd0..14c64bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,9 +33,6 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -123,8 +120,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-
-  @Test
+  
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -148,7 +144,6 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
-  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -187,7 +182,6 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
-  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 5096192..52b0e70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,33 +23,23 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.Test;
+import org.apache.hadoop.mapreduce.*;
 
 import java.io.IOException;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
-  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
-  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 19b712f..49b59ca 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,11 +33,6 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -59,15 +54,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Before
-  public void setUp() throws Exception {
+  @Override
+  protected void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @After
-  public void tearDown() throws Exception {
+  @Override
+  protected void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -224,7 +219,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -244,7 +238,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
-  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -271,7 +264,6 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
-  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index babd20e..6c432dd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,36 +27,23 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.CounterGroup;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.MapReduceTestUtil;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.hadoop.mapreduce.*;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
-  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
-  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -70,7 +57,6 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
-  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -78,7 +64,6 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
-  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 0d75d2f..3a2b831 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
+import java.io.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -30,15 +32,6 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -52,8 +45,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-
-  private void testComparator(String keySpec, int expect)
+  
+  private void testComparator(String keySpec, int expect) 
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -100,8 +93,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-
-  @Test
+  
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -125,7 +117,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6876b9f9/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 391e482..3492089 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,12 +27,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
-import org.junit.After;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -41,8 +35,7 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  @After
-  public void tearDown() throws Exception {
+  protected void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -83,7 +76,6 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
-  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -112,7 +104,6 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
-  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[05/20] hadoop git commit: HDFS-9927. Document the new OIV ReverseXML processor (Wei-Chiu Chuang via cmccabe)

Posted by ar...@apache.org.
HDFS-9927. Document the new OIV ReverseXML processor (Wei-Chiu Chuang via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 500875dfccdb3bb6709767962d1927ddb1cc5514
Parents: 017d2c1
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 10 13:38:00 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 10 13:38:00 2016 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HdfsImageViewer.md | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/500875df/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
index 77219d3..de27fc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
@@ -38,7 +38,7 @@ The Offline Image Viewer provides several output processors:
     interactively by using HTTP REST API.
 
 2.  XML creates an XML document of the fsimage and includes all of the
-    information within the fsimage, similar to the lsr processor. The
+    information within the fsimage. The
     output of this processor is amenable to automated processing and
     analysis with XML tools. Due to the verbosity of the XML syntax,
     this processor will also generate the largest amount of output.
@@ -60,6 +60,11 @@ The Offline Image Viewer provides several output processors:
    delimiter. The default delimiter is \t, though this may be changed via
    the -delimiter argument.
 
+5. ReverseXML (experimental): This is the opposite of the XML processor;
+   it reconstructs an fsimage from an XML file. This processor makes it easy to
+   create fsimages for testing, and manually edit fsimages when there is
+   corruption.
+
 Usage
 -----
 
@@ -139,9 +144,9 @@ Options
 
 | **Flag** | **Description** |
 |:---- |:---- |
-| `-i`\|`--inputFile` *input file* | Specify the input fsimage file to process. Required. |
-| `-o`\|`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. (output to stdout by default)\|
-| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Currently valid options are Web (default), XML, Delimited and FileDistribution. |
+| `-i`\|`--inputFile` *input file* | Specify the input fsimage file (or XML file, if ReverseXML processor is used) to process. Required. |
+| `-o`\|`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. (output to stdout by default) If the input file is an XML file, it also creates an &lt;outputFile&gt;.md5. |
+| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Currently valid options are `Web` (default), `XML`, `Delimited`, `FileDistribution` and `ReverseXML`. |
 | `-addr` *address* | Specify the address(host:port) to listen. (localhost:5978 by default). This option is used with Web processor. |
 | `-maxSize` *size* | Specify the range [0, maxSize] of file sizes to be analyzed in bytes (128GB by default). This option is used with FileDistribution processor. |
 | `-step` *size* | Specify the granularity of the distribution in bytes (2MB by default). This option is used with FileDistribution processor. |
@@ -157,7 +162,7 @@ The Offline Image Viewer makes it easy to gather large amounts of data about the
 oiv\_legacy Command
 -------------------
 
-Due to the internal layout changes introduced by the ProtocolBuffer-based fsimage ([HDFS-5698](https://issues.apache.org/jira/browse/HDFS-5698)), OfflineImageViewer consumes excessive amount of memory and loses some functions such as Indented and Delimited processor. If you want to process without large amount of memory or use these processors, you can use `oiv_legacy` command (same as `oiv` in Hadoop 2.3).
+Due to the internal layout changes introduced by the ProtocolBuffer-based fsimage ([HDFS-5698](https://issues.apache.org/jira/browse/HDFS-5698)), OfflineImageViewer consumes excessive amount of memory and loses some functions such as Indented processor. If you want to process without large amount of memory or use these processors, you can use `oiv_legacy` command (same as `oiv` in Hadoop 2.3).
 
 ### Usage
 


[17/20] hadoop git commit: HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)

Posted by ar...@apache.org.
HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 658ee95ff367eaf58d76581e75bdaba0dd0d871e
Parents: 6e9a582
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 11 12:56:12 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 11 12:57:54 2016 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/CMakeLists.txt            |  22 +-
 .../src/main/native/native.vcxproj              |  11 +-
 .../io/erasurecode/coder/erasure_code_native.c  |  49 ---
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 --
 .../src/org/apache/hadoop/io/erasurecode/dump.c | 100 ++++++
 .../src/org/apache/hadoop/io/erasurecode/dump.h |  40 +++
 .../apache/hadoop/io/erasurecode/erasure_code.c | 239 +--------------
 .../apache/hadoop/io/erasurecode/erasure_code.h | 107 +++++++
 .../hadoop/io/erasurecode/erasure_coder.c       | 229 ++++++++++++++
 .../hadoop/io/erasurecode/erasure_coder.h       |  88 ++++++
 .../org/apache/hadoop/io/erasurecode/gf_util.c  |  54 ++++
 .../org/apache/hadoop/io/erasurecode/gf_util.h  | 111 +++++++
 .../io/erasurecode/include/erasure_code.h       | 125 --------
 .../hadoop/io/erasurecode/include/gf_util.h     | 111 -------
 .../apache/hadoop/io/erasurecode/isal_load.c    | 148 +++++++++
 .../apache/hadoop/io/erasurecode/isal_load.h    | 149 +++++++++
 .../apache/hadoop/io/erasurecode/jni_common.c   |  98 ++++++
 .../apache/hadoop/io/erasurecode/jni_common.h   |  42 +++
 .../io/erasurecode/jni_erasure_code_native.c    |  45 +++
 .../hadoop/io/erasurecode/jni_rs_decoder.c      |  72 +++++
 .../hadoop/io/erasurecode/jni_rs_encoder.c      |  66 ++++
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 ++
 ...io_erasurecode_rawcoder_NativeRSRawDecoder.h |  37 +++
 ...io_erasurecode_rawcoder_NativeRSRawEncoder.h |  37 +++
 .../hadoop/io/erasurecode/erasure_code_test.c   | 307 ++++---------------
 25 files changed, 1545 insertions(+), 800 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 51fb0fe..8026eb5 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -103,13 +103,25 @@ find_library(ISAL_LIBRARY
 set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
 if (ISAL_LIBRARY)
     GET_FILENAME_COMPONENT(HADOOP_ISAL_LIBRARY ${ISAL_LIBRARY} NAME)
-    set(ISAL_INCLUDE_DIR ${SRC}/io/erasurecode/include)
+    set(ISAL_INCLUDE_DIR ${SRC}/io/erasurecode)
     set(ISAL_SOURCE_FILES
-        ${SRC}/io/erasurecode/coder/erasure_code_native.c
-        ${SRC}/io/erasurecode/erasure_code.c)
+        ${SRC}/io/erasurecode/isal_load.c
+        ${SRC}/io/erasurecode/erasure_code.c
+        ${SRC}/io/erasurecode/gf_util.c
+        ${SRC}/io/erasurecode/dump.c
+        ${SRC}/io/erasurecode/erasure_coder.c
+        ${SRC}/io/erasurecode/jni_erasure_code_native.c
+        ${SRC}/io/erasurecode/jni_common.c
+        ${SRC}/io/erasurecode/jni_rs_encoder.c
+        ${SRC}/io/erasurecode/jni_rs_decoder.c)
+
         add_executable(erasure_code_test
-             ${SRC}/io/erasurecode/erasure_code.c
-             ${TST}/io/erasurecode/erasure_code_test.c
+        ${SRC}/io/erasurecode/isal_load.c
+        ${SRC}/io/erasurecode/erasure_code.c
+        ${SRC}/io/erasurecode/gf_util.c
+        ${SRC}/io/erasurecode/dump.c
+        ${SRC}/io/erasurecode/erasure_coder.c
+        ${TST}/io/erasurecode/erasure_code_test.c
         )
         target_link_libraries(erasure_code_test ${CMAKE_DL_LIBS})
 else (ISAL_LIBRARY)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
index 17149f7..2274c41 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
@@ -162,10 +162,17 @@
     <ClCompile Include="src\org\apache\hadoop\yarn\server\nodemanager\windows_secure_container_executor.c">
       <AdditionalIncludeDirectories>src\org\apache\hadoop\io\nativeio;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
     </ClCompile>
-    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_code.c" Condition="'$(IsalEnabled)' == 'true'">
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\isal_load.c" Condition="'$(IsalEnabled)' == 'true'">
       <AdditionalOptions>/D HADOOP_ISAL_LIBRARY=\"isa-l.dll\"</AdditionalOptions>
     </ClCompile>
-    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\coder\erasure_code_native.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_code.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\gf_util.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\erasure_coder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\dump.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_erasure_code_native.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_common.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_encoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
+    <ClCompile Include="src\org\apache\hadoop\io\erasurecode\jni_rs_decoder.c" Condition="'$(IsalEnabled)' == 'true'"/>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\src\org\apache\hadoop\util\crc32c_tables.h" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
deleted file mode 100644
index e84df9a..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.
- */
-
-#include <stdio.h>
-#include <stdlib.h>
-#include <string.h>
-
-#include "org_apache_hadoop.h"
-#include "../include/erasure_code.h"
-#include "org_apache_hadoop_io_erasurecode_ErasureCodeNative.h"
-
-#ifdef UNIX
-#include "config.h"
-#endif
-
-JNIEXPORT void JNICALL
-Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
-(JNIEnv *env, jclass myclass) {
-  char errMsg[1024];
-  load_erasurecode_lib(errMsg, sizeof(errMsg));
-  if (strlen(errMsg) > 0) {
-    THROW(env, "java/lang/UnsatisfiedLinkError", errMsg);
-  }
-}
-
-JNIEXPORT jstring JNICALL
-Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
-(JNIEnv *env, jclass myclass) {
-  char* libName = get_library_name();
-  if (libName == NULL) {
-    libName = "Unavailable";
-  }
-  return (*env)->NewStringUTF(env, libName);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
deleted file mode 100644
index d8ff3a0..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
+++ /dev/null
@@ -1,29 +0,0 @@
-/* DO NOT EDIT THIS FILE - it is machine generated */
-#include <jni.h>
-/* Header for class org_apache_hadoop_io_erasurecode_ErasureCodeNative */
-
-#ifndef _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
-#define _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
-#ifdef __cplusplus
-extern "C" {
-#endif
-/*
- * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
- * Method:    loadLibrary
- * Signature: ()V
- */
-JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
-  (JNIEnv *, jclass);
-
-/*
- * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
- * Method:    getLibraryName
- * Signature: ()Ljava/lang/String;
- */
-JNIEXPORT jstring JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
-  (JNIEnv *, jclass);
-
-#ifdef __cplusplus
-}
-#endif
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
new file mode 100644
index 0000000..20bd189
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
@@ -0,0 +1,100 @@
+/**
+ * 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.
+ */
+
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "erasure_coder.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void dump(unsigned char* buf, int len) {
+  int i;
+  for (i = 0; i < len;) {
+    printf(" %2x", 0xff & buf[i++]);
+    if (i % 32 == 0)
+      printf("\n");
+  }
+}
+
+void dumpMatrix(unsigned char** buf, int n1, int n2) {
+  int i, j;
+  for (i = 0; i < n1; i++) {
+    for (j = 0; j < n2; j++) {
+      printf(" %2x", buf[i][j]);
+    }
+    printf("\n");
+  }
+  printf("\n");
+}
+
+void dumpCodingMatrix(unsigned char* buf, int n1, int n2) {
+  int i, j;
+  for (i = 0; i < n1; i++) {
+    for (j = 0; j < n2; j++) {
+      printf(" %d", 0xff & buf[j + (i * n2)]);
+    }
+    printf("\n");
+  }
+  printf("\n");
+}
+
+void dumpEncoder(IsalEncoder* pCoder) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numParityUnits = pCoder->coder.numDataUnits;
+  int numAllUnits = pCoder->coder.numAllUnits;
+
+  printf("Encoding (numAlnumParityUnitslUnits = %d, numDataUnits = %d)\n",
+                                    numParityUnits, numDataUnits);
+
+  printf("\n\nEncodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->encodeMatrix,
+                                           numDataUnits, numAllUnits);
+}
+
+void dumpDecoder(IsalDecoder* pCoder) {
+  int i, j;
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numAllUnits = pCoder->coder.numAllUnits;
+
+  printf("Recovering (numAllUnits = %d, numDataUnits = %d, numErased = %d)\n",
+                       numAllUnits, numDataUnits, pCoder->numErased);
+
+  printf(" - ErasedIndexes = ");
+  for (j = 0; j < pCoder->numErased; j++) {
+    printf(" %d", pCoder->erasedIndexes[j]);
+  }
+  printf("       - DecodeIndex = ");
+  for (i = 0; i < numDataUnits; i++) {
+    printf(" %d", pCoder->decodeIndex[i]);
+  }
+
+  printf("\n\nEncodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->encodeMatrix,
+                                    numDataUnits, numAllUnits);
+
+  printf("InvertMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->invertMatrix,
+                                   numDataUnits, numDataUnits);
+
+  printf("DecodeMatrix:\n");
+  dumpCodingMatrix((unsigned char*) pCoder->decodeMatrix,
+                                    numDataUnits, numAllUnits);
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
new file mode 100644
index 0000000..f58d960
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
@@ -0,0 +1,40 @@
+/**
+ * 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.
+ */
+
+/**
+ * Dump utilities for erasure coders.
+ */
+
+#ifndef _DUMP_H_
+#define _DUMP_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void dumpEncoder(IsalEncoder* pCoder);
+
+void dumpDecoder(IsalDecoder* pCoder);
+
+void dump(unsigned char* buf, int len);
+
+void dumpMatrix(unsigned char** s, int k, int m);
+
+void dumpCodingMatrix(unsigned char* s, int n1, int n2);
+
+#endif //_DUMP_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
index a6c099a..1250688 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
@@ -20,252 +20,25 @@
 #include <stdlib.h>
 #include <string.h>
 
-#include "org_apache_hadoop.h"
-#include "../include/gf_util.h"
-#include "../include/erasure_code.h"
-
-#ifdef UNIX
-#include <sys/time.h>
-#include <sys/types.h>
-#include <sys/stat.h>
-#include <dlfcn.h>
-
-#include "config.h"
-#endif
-
-#ifdef WINDOWS
-#include <Windows.h>
-#endif
+#include "isal_load.h"
+#include "erasure_code.h"
 
 /**
  *  erasure_code.c
- *  Implementation erasure code utilities based on lib of erasure_code.so.
- *  Building of this codes won't rely on any ISA-L source codes, but running
- *  into this will rely on successfully loading of the dynamic library.
+ *  Implementation erasure code utilities based on ISA-L library.
  *
  */
 
-/**
- * The loaded library handle.
- */
-static void* libec = NULL;
-
-/**
- * A helper function to dlsym a 'symbol' from a given library-handle.
- */
-
-#ifdef UNIX
-
-static __attribute__ ((unused))
-void *my_dlsym(void *handle, const char *symbol) {
-  void *func_ptr = dlsym(handle, symbol);
-  return func_ptr;
-}
-
-/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
-#define EC_LOAD_DYNAMIC_SYMBOL(func_ptr, handle, symbol) \
-  if ((func_ptr = my_dlsym(handle, symbol)) == NULL) { \
-    return "Failed to load symbol" symbol; \
-  }
-
-#endif
-
-#ifdef WINDOWS
-
-
-
-static FARPROC WINAPI my_dlsym(HMODULE handle, LPCSTR symbol) {
-  FARPROC func_ptr = GetProcAddress(handle, symbol);
-  return func_ptr;
-}
-
-/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
-#define EC_LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, handle, symbol) \
-  if ((func_ptr = (func_type)my_dlsym(handle, symbol)) == NULL) { \
-    return "Failed to load symbol" symbol; \
-  }
-
-#endif
-
-
-#ifdef UNIX
-// For gf_util.h
-static unsigned char (*d_gf_mul)(unsigned char, unsigned char);
-static unsigned char (*d_gf_inv)(unsigned char);
-static void (*d_gf_gen_rs_matrix)(unsigned char *, int, int);
-static void (*d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
-static int (*d_gf_invert_matrix)(unsigned char *, unsigned char *, const int);
-static int (*d_gf_vect_mul)(int, unsigned char *, void *, void *);
-
-// For erasure_code.h
-static void (*d_ec_init_tables)(int, int, unsigned char*, unsigned char*);
-static void (*d_ec_encode_data)(int, int, int, unsigned char*,
-                                          unsigned char**, unsigned char**);
-static void (*d_ec_encode_data_update)(int, int, int, int, unsigned char*,
-                                             unsigned char*, unsigned char**);
-#endif
-
-#ifdef WINDOWS
-// For erasure_code.h
-typedef unsigned char (__cdecl *__d_gf_mul)(unsigned char, unsigned char);
-static __d_gf_mul d_gf_mul;
-typedef unsigned char (__cdecl *__d_gf_inv)(unsigned char);
-static __d_gf_inv d_gf_inv;
-typedef void (__cdecl *__d_gf_gen_rs_matrix)(unsigned char *, int, int);
-static __d_gf_gen_rs_matrix d_gf_gen_rs_matrix;
-typedef void (__cdecl *__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
-static __d_gf_gen_cauchy_matrix d_gf_gen_cauchy_matrix;
-typedef int (__cdecl *__d_gf_invert_matrix)(unsigned char *,
-                                                   unsigned char *, const int);
-static __d_gf_invert_matrix d_gf_invert_matrix;
-typedef int (__cdecl *__d_gf_vect_mul)(int, unsigned char *, void *, void *);
-static __d_gf_vect_mul d_gf_vect_mul;
-
-// For erasure_code.h
-typedef void (__cdecl *__d_ec_init_tables)(int, int,
-                                                unsigned char*, unsigned char*);
-static __d_ec_init_tables d_ec_init_tables;
-typedef void (__cdecl *__d_ec_encode_data)(int, int, int, unsigned char*,
-                                             unsigned char**, unsigned char**);
-static __d_ec_encode_data d_ec_encode_data;
-typedef void (__cdecl *__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
-                                             unsigned char*, unsigned char**);
-static __d_ec_encode_data_update d_ec_encode_data_update;
-#endif
-
-static const char* load_functions(void* libec) {
-#ifdef UNIX
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_mul, libec, "gf_mul");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_inv, libec, "gf_inv");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_gen_rs_matrix, libec, "gf_gen_rs_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_gen_cauchy_matrix, libec, "gf_gen_cauchy1_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_invert_matrix, libec, "gf_invert_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(d_gf_vect_mul, libec, "gf_vect_mul");
-
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_init_tables, libec, "ec_init_tables");
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_encode_data, libec, "ec_encode_data");
-  EC_LOAD_DYNAMIC_SYMBOL(d_ec_encode_data_update, libec, "ec_encode_data_update");
-#endif
-
-#ifdef WINDOWS
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_mul, d_gf_mul, libec, "gf_mul");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_inv, d_gf_inv, libec, "gf_inv");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_rs_matrix, d_gf_gen_rs_matrix, libec, "gf_gen_rs_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_cauchy_matrix, d_gf_gen_cauchy_matrix, libec, "gf_gen_cauchy1_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_invert_matrix, d_gf_invert_matrix, libec, "gf_invert_matrix");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_vect_mul, d_gf_vect_mul, libec, "gf_vect_mul");
-
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_init_tables, d_ec_init_tables, libec, "ec_init_tables");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data, d_ec_encode_data, libec, "ec_encode_data");
-  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data_update, d_ec_encode_data_update, libec, "ec_encode_data_update");
-#endif
-
-  return NULL;
-}
-
-void load_erasurecode_lib(char* err, size_t err_len) {
-  const char* errMsg;
-
-  err[0] = '\0';
-
-  if (libec != NULL) {
-    return;
-  }
-
-  // Load Intel ISA-L
-  #ifdef UNIX
-  libec = dlopen(HADOOP_ISAL_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
-  if (libec == NULL) {
-    snprintf(err, err_len, "Failed to load %s (%s)",
-                             HADOOP_ISAL_LIBRARY, dlerror());
-    return;
-  }
-  // Clear any existing error
-  dlerror();
-  #endif
-
-  #ifdef WINDOWS
-  libec = LoadLibrary(HADOOP_ISAL_LIBRARY);
-  if (libec == NULL) {
-    snprintf(err, err_len, "Failed to load %s", HADOOP_ISAL_LIBRARY);
-    return;
-  }
-  #endif
-
-  errMsg = load_functions(libec);
-  if (errMsg != NULL) {
-    snprintf(err, err_len, "Loading functions from ISA-L failed: %s", errMsg);
-  }
-}
-
-int build_support_erasurecode() {
-#ifdef HADOOP_ISAL_LIBRARY
-  return 1;
-#else
-  return 0;
-#endif
-}
-
-const char* get_library_name() {
-#ifdef UNIX
-  Dl_info dl_info;
-
-  if (d_ec_encode_data == NULL) {
-    return HADOOP_ISAL_LIBRARY;
-  }
-
-  if(dladdr(d_ec_encode_data, &dl_info)) {
-    return dl_info.dli_fname;
-  }
-#else
-  LPTSTR filename = NULL;
-
-  if (libec == NULL) {
-    return HADOOP_ISAL_LIBRARY;
-  }
-
-  if (GetModuleFileName(libec, filename, 256) > 0) {
-    return filename;
-  }
-#endif
-
-  return NULL;
-}
-
-unsigned char h_gf_mul(unsigned char a, unsigned char b) {
-  return d_gf_mul(a, b);
-}
-
-unsigned char h_gf_inv(unsigned char a) {
-  return d_gf_inv(a);
-}
-
-void h_gf_gen_rs_matrix(unsigned char *a, int m, int k) {
-  d_gf_gen_rs_matrix(a, m, k);
-}
-
-void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k) {
-  d_gf_gen_cauchy_matrix(a, m, k);
-}
-
-int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n) {
-  return d_gf_invert_matrix(in, out, n);
-}
-
-int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest) {
-  return d_gf_vect_mul(len, gftbl, src, dest);
-}
-
 void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls) {
-  d_ec_init_tables(k, rows, a, gftbls);
+  isaLoader->ec_init_tables(k, rows, a, gftbls);
 }
 
 void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
     unsigned char **data, unsigned char **coding) {
-  d_ec_encode_data(len, k, rows, gftbls, data, coding);
+  isaLoader->ec_encode_data(len, k, rows, gftbls, data, coding);
 }
 
 void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
          unsigned char *gftbls, unsigned char *data, unsigned char **coding) {
-  d_ec_encode_data_update(len, k, rows, vec_i, gftbls, data, coding);
+  isaLoader->ec_encode_data_update(len, k, rows, vec_i, gftbls, data, coding);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
new file mode 100644
index 0000000..a6cfd58
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
@@ -0,0 +1,107 @@
+/*
+ *  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.
+ */
+
+#ifndef _ERASURE_CODE_H_
+#define _ERASURE_CODE_H_
+
+#include <stddef.h>
+
+/**
+ *  Interface to functions supporting erasure code encode and decode.
+ *
+ *  This file defines the interface to optimized functions used in erasure
+ *  codes.  Encode and decode of erasures in GF(2^8) are made by calculating the
+ *  dot product of the symbols (bytes in GF(2^8)) across a set of buffers and a
+ *  set of coefficients.  Values for the coefficients are determined by the type
+ *  of erasure code.  Using a general dot product means that any sequence of
+ *  coefficients may be used including erasure codes based on random
+ *  coefficients.
+ *  Multiple versions of dot product are supplied to calculate 1-6 output
+ *  vectors in one pass.
+ *  Base GF multiply and divide functions can be sped up by defining
+ *  GF_LARGE_TABLES at the expense of memory size.
+ *
+ */
+
+/**
+ * Initialize tables for fast Erasure Code encode and decode.
+ *
+ * Generates the expanded tables needed for fast encode or decode for erasure
+ * codes on blocks of data.  32bytes is generated for each input coefficient.
+ *
+ * @param k      The number of vector sources or rows in the generator matrix
+ *               for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param a      Pointer to sets of arrays of input coefficients used to encode
+ *               or decode data.
+ * @param gftbls Pointer to start of space for concatenated output tables
+ *               generated from input coefficients.  Must be of size 32*k*rows.
+ * @returns none
+ */
+void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls);
+
+/**
+ * Generate or decode erasure codes on blocks of data, runs appropriate version.
+ *
+ * Given a list of source data blocks, generate one or multiple blocks of
+ * encoded data as specified by a matrix of GF(2^8) coefficients. When given a
+ * suitable set of coefficients, this function will perform the fast generation
+ * or decoding of Reed-Solomon type erasure codes.
+ *
+ * This function determines what instruction sets are enabled and
+ * selects the appropriate version at runtime.
+ *
+ * @param len    Length of each block of data (vector) of source or dest data.
+ * @param k      The number of vector sources or rows in the generator matrix
+ *        for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param gftbls Pointer to array of input tables generated from coding
+ *        coefficients in ec_init_tables(). Must be of size 32*k*rows
+ * @param data   Array of pointers to source input buffers.
+ * @param coding Array of pointers to coded output buffers.
+ * @returns none
+ */
+void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
+                                 unsigned char **data, unsigned char **coding);
+
+/**
+ * @brief Generate update for encode or decode of erasure codes from single
+ *        source, runs appropriate version.
+ *
+ * Given one source data block, update one or multiple blocks of encoded data as
+ * specified by a matrix of GF(2^8) coefficients. When given a suitable set of
+ * coefficients, this function will perform the fast generation or decoding of
+ * Reed-Solomon type erasure codes from one input source at a time.
+ *
+ * This function determines what instruction sets are enabled and selects the
+ * appropriate version at runtime.
+ *
+ * @param len    Length of each block of data (vector) of source or dest data.
+ * @param k      The number of vector sources or rows in the generator matrix
+ *               for coding.
+ * @param rows   The number of output vectors to concurrently encode/decode.
+ * @param vec_i  The vector index corresponding to the single input source.
+ * @param gftbls Pointer to array of input tables generated from coding
+ *               coefficients in ec_init_tables(). Must be of size 32*k*rows
+ * @param data   Pointer to single input source used to update output parity.
+ * @param coding Array of pointers to coded output buffers.
+ * @returns none
+ */
+void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
+           unsigned char *gftbls, unsigned char *data, unsigned char **coding);
+
+#endif //_ERASURE_CODE_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
new file mode 100644
index 0000000..b3479bb
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
@@ -0,0 +1,229 @@
+/**
+ * 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.
+ */
+
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "erasure_coder.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+void initCoder(IsalCoder* pCoder, int numDataUnits, int numParityUnits) {
+  pCoder->verbose = 0;
+  pCoder->numParityUnits = numParityUnits;
+  pCoder->numDataUnits = numDataUnits;
+  pCoder->numAllUnits = numDataUnits + numParityUnits;
+}
+
+// 0 not to verbose, 1 to verbose
+void allowVerbose(IsalCoder* pCoder, int flag) {
+  pCoder->verbose = flag;
+}
+
+static void initEncodeMatrix(int numDataUnits, int numParityUnits,
+                                                unsigned char* encodeMatrix) {
+  // Generate encode matrix, always invertible
+  h_gf_gen_cauchy_matrix(encodeMatrix,
+                          numDataUnits + numParityUnits, numDataUnits);
+}
+
+void initEncoder(IsalEncoder* pCoder, int numDataUnits,
+                            int numParityUnits) {
+  initCoder(&pCoder->coder, numDataUnits, numParityUnits);
+
+  initEncodeMatrix(numDataUnits, numParityUnits, pCoder->encodeMatrix);
+
+  // Generate gftbls from encode matrix
+  h_ec_init_tables(numDataUnits, numParityUnits,
+               &pCoder->encodeMatrix[numDataUnits * numDataUnits],
+               pCoder->gftbls);
+
+  if (pCoder->coder.verbose > 0) {
+    dumpEncoder(pCoder);
+  }
+}
+
+void initDecoder(IsalDecoder* pCoder, int numDataUnits,
+                                  int numParityUnits) {
+  initCoder(&pCoder->coder, numDataUnits, numParityUnits);
+
+  initEncodeMatrix(numDataUnits, numParityUnits, pCoder->encodeMatrix);
+}
+
+int encode(IsalEncoder* pCoder, unsigned char** dataUnits,
+    unsigned char** parityUnits, int chunkSize) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int numParityUnits = pCoder->coder.numParityUnits;
+  int i;
+
+  for (i = 0; i < numParityUnits; i++) {
+    memset(parityUnits[i], 0, chunkSize);
+  }
+
+  h_ec_encode_data(chunkSize, numDataUnits, numParityUnits,
+                         pCoder->gftbls, dataUnits, parityUnits);
+
+  return 0;
+}
+
+// Return 1 when diff, 0 otherwise
+static int compare(int* arr1, int len1, int* arr2, int len2) {
+  int i;
+
+  if (len1 == len2) {
+    for (i = 0; i < len1; i++) {
+      if (arr1[i] != arr2[i]) {
+        return 1;
+      }
+    }
+    return 0;
+  }
+
+  return 1;
+}
+
+static int processErasures(IsalDecoder* pCoder, unsigned char** inputs,
+                                    int* erasedIndexes, int numErased) {
+  int i, r, ret, index;
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int isChanged = 0;
+
+  for (i = 0, r = 0; i < numDataUnits; i++, r++) {
+    while (inputs[r] == NULL) {
+      r++;
+    }
+
+    if (pCoder->decodeIndex[i] != r) {
+      pCoder->decodeIndex[i] = r;
+      isChanged = 1;
+    }
+  }
+
+  for (i = 0; i < numDataUnits; i++) {
+    pCoder->realInputs[i] = inputs[pCoder->decodeIndex[i]];
+  }
+
+  if (isChanged == 0 &&
+          compare(pCoder->erasedIndexes, pCoder->numErased,
+                           erasedIndexes, numErased) == 0) {
+    return 0; // Optimization, nothing to do
+  }
+
+  clearDecoder(pCoder);
+
+  for (i = 0; i < numErased; i++) {
+    index = erasedIndexes[i];
+    pCoder->erasedIndexes[i] = index;
+    pCoder->erasureFlags[index] = 1;
+    if (index < numDataUnits) {
+      pCoder->numErasedDataUnits++;
+    }
+  }
+
+  pCoder->numErased = numErased;
+
+  ret = generateDecodeMatrix(pCoder);
+  if (ret != 0) {
+    printf("Failed to generate decode matrix\n");
+    return -1;
+  }
+
+  h_ec_init_tables(numDataUnits, pCoder->numErased,
+                      pCoder->decodeMatrix, pCoder->gftbls);
+
+  if (pCoder->coder.verbose > 0) {
+    dumpDecoder(pCoder);
+  }
+
+  return 0;
+}
+
+int decode(IsalDecoder* pCoder, unsigned char** inputs,
+                  int* erasedIndexes, int numErased,
+                   unsigned char** outputs, int chunkSize) {
+  int numDataUnits = pCoder->coder.numDataUnits;
+  int i;
+
+  processErasures(pCoder, inputs, erasedIndexes, numErased);
+
+  for (i = 0; i < numErased; i++) {
+    memset(outputs[i], 0, chunkSize);
+  }
+
+  h_ec_encode_data(chunkSize, numDataUnits, pCoder->numErased,
+      pCoder->gftbls, pCoder->realInputs, outputs);
+
+  return 0;
+}
+
+// Clear variables used per decode call
+void clearDecoder(IsalDecoder* decoder) {
+  decoder->numErasedDataUnits = 0;
+  decoder->numErased = 0;
+  memset(decoder->gftbls, 0, sizeof(decoder->gftbls));
+  memset(decoder->decodeMatrix, 0, sizeof(decoder->decodeMatrix));
+  memset(decoder->tmpMatrix, 0, sizeof(decoder->tmpMatrix));
+  memset(decoder->invertMatrix, 0, sizeof(decoder->invertMatrix));
+  memset(decoder->erasureFlags, 0, sizeof(decoder->erasureFlags));
+  memset(decoder->erasedIndexes, 0, sizeof(decoder->erasedIndexes));
+}
+
+// Generate decode matrix from encode matrix
+int generateDecodeMatrix(IsalDecoder* pCoder) {
+  int i, j, r, p;
+  unsigned char s;
+  int numDataUnits;
+
+  numDataUnits = pCoder->coder.numDataUnits;
+
+  // Construct matrix b by removing error rows
+  for (i = 0; i < numDataUnits; i++) {
+    r = pCoder->decodeIndex[i];
+    for (j = 0; j < numDataUnits; j++) {
+      pCoder->tmpMatrix[numDataUnits * i + j] =
+                pCoder->encodeMatrix[numDataUnits * r + j];
+    }
+  }
+
+  h_gf_invert_matrix(pCoder->tmpMatrix,
+                                pCoder->invertMatrix, numDataUnits);
+
+  for (i = 0; i < pCoder->numErasedDataUnits; i++) {
+    for (j = 0; j < numDataUnits; j++) {
+      pCoder->decodeMatrix[numDataUnits * i + j] =
+                      pCoder->invertMatrix[numDataUnits *
+                      pCoder->erasedIndexes[i] + j];
+    }
+  }
+
+  for (p = pCoder->numErasedDataUnits; p < pCoder->numErased; p++) {
+    for (i = 0; i < numDataUnits; i++) {
+      s = 0;
+      for (j = 0; j < numDataUnits; j++) {
+        s ^= h_gf_mul(pCoder->invertMatrix[j * numDataUnits + i],
+          pCoder->encodeMatrix[numDataUnits *
+                                        pCoder->erasedIndexes[p] + j]);
+      }
+
+      pCoder->decodeMatrix[numDataUnits * p + i] = s;
+    }
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
new file mode 100644
index 0000000..8f5bf8a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+/**
+ * This is a sample program illustrating how to use the Intel ISA-L library.
+ * Note it's adapted from erasure_code_test.c test program, but trying to use
+ * variable names and styles we're more familiar with already similar to Java
+ * coders.
+ */
+
+#ifndef _ERASURE_CODER_H_
+#define _ERASURE_CODER_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define MMAX 14
+#define KMAX 10
+
+typedef struct _IsalCoder {
+  int verbose;
+  int numParityUnits;
+  int numDataUnits;
+  int numAllUnits;
+} IsalCoder;
+
+typedef struct _IsalEncoder {
+  IsalCoder coder;
+
+  unsigned char gftbls[MMAX * KMAX * 32];
+
+  unsigned char encodeMatrix[MMAX * KMAX];
+} IsalEncoder;
+
+typedef struct _IsalDecoder {
+  IsalCoder coder;
+
+  unsigned char encodeMatrix[MMAX * KMAX];
+
+  // Below are per decode call
+  unsigned char gftbls[MMAX * KMAX * 32];
+  unsigned int decodeIndex[MMAX];
+  unsigned char tmpMatrix[MMAX * KMAX];
+  unsigned char invertMatrix[MMAX * KMAX];
+  unsigned char decodeMatrix[MMAX * KMAX];
+  unsigned char erasureFlags[MMAX];
+  int erasedIndexes[MMAX];
+  int numErased;
+  int numErasedDataUnits;
+  unsigned char* realInputs[MMAX];
+} IsalDecoder;
+
+void initCoder(IsalCoder* pCoder, int numDataUnits, int numParityUnits);
+
+void allowVerbose(IsalCoder* pCoder, int flag);
+
+void initEncoder(IsalEncoder* encoder, int numDataUnits, int numParityUnits);
+
+void initDecoder(IsalDecoder* decoder, int numDataUnits, int numParityUnits);
+
+void clearDecoder(IsalDecoder* decoder);
+
+int encode(IsalEncoder* encoder, unsigned char** dataUnits,
+    unsigned char** parityUnits, int chunkSize);
+
+int decode(IsalDecoder* decoder, unsigned char** allUnits,
+    int* erasedIndexes, int numErased,
+    unsigned char** recoveredUnits, int chunkSize);
+
+int generateDecodeMatrix(IsalDecoder* pCoder);
+
+#endif //_ERASURE_CODER_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
new file mode 100644
index 0000000..804dcec
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
@@ -0,0 +1,54 @@
+/*
+ *  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.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "isal_load.h"
+#include "gf_util.h"
+
+/**
+ *  gf_util.c
+ *  Implementation GF utilities based on ISA-L library.
+ *
+ */
+
+unsigned char h_gf_mul(unsigned char a, unsigned char b) {
+  return isaLoader->gf_mul(a, b);
+}
+
+unsigned char h_gf_inv(unsigned char a) {
+  return isaLoader->gf_inv(a);
+}
+
+void h_gf_gen_rs_matrix(unsigned char *a, int m, int k) {
+  isaLoader->gf_gen_rs_matrix(a, m, k);
+}
+
+void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k) {
+  isaLoader->gf_gen_cauchy_matrix(a, m, k);
+}
+
+int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n) {
+  return isaLoader->gf_invert_matrix(in, out, n);
+}
+
+int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest) {
+  return isaLoader->gf_vect_mul(len, gftbl, src, dest);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
new file mode 100644
index 0000000..2be8328
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
@@ -0,0 +1,111 @@
+/*
+ *  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.
+ */
+
+#ifndef _GF_UTIL_H
+#define _GF_UTIL_H
+
+/**
+ *  gf_util.h
+ *  Interface to functions for vector (block) multiplication in GF(2^8).
+ *
+ *  This file defines the interface to routines used in fast RAID rebuild and
+ *  erasure codes.
+ */
+
+
+/**
+ * Single element GF(2^8) multiply.
+ *
+ * @param a  Multiplicand a
+ * @param b  Multiplicand b
+ * @returns  Product of a and b in GF(2^8)
+ */
+unsigned char h_gf_mul(unsigned char a, unsigned char b);
+
+/**
+ * Single element GF(2^8) inverse.
+ *
+ * @param a  Input element
+ * @returns  Field element b such that a x b = {1}
+ */
+unsigned char h_gf_inv(unsigned char a);
+
+/**
+ * Generate a matrix of coefficients to be used for encoding.
+ *
+ * Vandermonde matrix example of encoding coefficients where high portion of
+ * matrix is identity matrix I and lower portion is constructed as 2^{i*(j-k+1)}
+ * i:{0,k-1} j:{k,m-1}. Commonly used method for choosing coefficients in
+ * erasure encoding but does not guarantee invertable for every sub matrix.  For
+ * large k it is possible to find cases where the decode matrix chosen from
+ * sources and parity not in erasure are not invertable. Users may want to
+ * adjust for k > 5.
+ *
+ * @param a  [mxk] array to hold coefficients
+ * @param m  number of rows in matrix corresponding to srcs + parity.
+ * @param k  number of columns in matrix corresponding to srcs.
+ * @returns  none
+ */
+void h_gf_gen_rs_matrix(unsigned char *a, int m, int k);
+
+/**
+ * Generate a Cauchy matrix of coefficients to be used for encoding.
+ *
+ * Cauchy matrix example of encoding coefficients where high portion of matrix
+ * is identity matrix I and lower portion is constructed as 1/(i + j) | i != j,
+ * i:{0,k-1} j:{k,m-1}.  Any sub-matrix of a Cauchy matrix should be invertable.
+ *
+ * @param a  [mxk] array to hold coefficients
+ * @param m  number of rows in matrix corresponding to srcs + parity.
+ * @param k  number of columns in matrix corresponding to srcs.
+ * @returns  none
+ */
+void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k);
+
+/**
+ * Invert a matrix in GF(2^8)
+ *
+ * @param in  input matrix
+ * @param out output matrix such that [in] x [out] = [I] - identity matrix
+ * @param n   size of matrix [nxn]
+ * @returns 0 successful, other fail on singular input matrix
+ */
+int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n);
+
+/**
+ * GF(2^8) vector multiply by constant, runs appropriate version.
+ *
+ * Does a GF(2^8) vector multiply b = Ca where a and b are arrays and C
+ * is a single field element in GF(2^8). Can be used for RAID6 rebuild
+ * and partial write functions. Function requires pre-calculation of a
+ * 32-element constant array based on constant C. gftbl(C) = {C{00},
+ * C{01}, C{02}, ... , C{0f} }, {C{00}, C{10}, C{20}, ... , C{f0} }.
+ * Len and src must be aligned to 32B.
+ *
+ * This function determines what instruction sets are enabled
+ * and selects the appropriate version at runtime.
+ *
+ * @param len   Length of vector in bytes. Must be aligned to 32B.
+ * @param gftbl Pointer to 32-byte array of pre-calculated constants based on C.
+ * @param src   Pointer to src data array. Must be aligned to 32B.
+ * @param dest  Pointer to destination data array. Must be aligned to 32B.
+ * @returns 0 pass, other fail
+ */
+int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest);
+
+
+#endif //_GF_UTIL_H

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
deleted file mode 100644
index 123085e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- *  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.
- */
-
-#ifndef _ERASURE_CODE_H_
-#define _ERASURE_CODE_H_
-
-#include <stddef.h>
-
-/**
- *  Interface to functions supporting erasure code encode and decode.
- *
- *  This file defines the interface to optimized functions used in erasure
- *  codes.  Encode and decode of erasures in GF(2^8) are made by calculating the
- *  dot product of the symbols (bytes in GF(2^8)) across a set of buffers and a
- *  set of coefficients.  Values for the coefficients are determined by the type
- *  of erasure code.  Using a general dot product means that any sequence of
- *  coefficients may be used including erasure codes based on random
- *  coefficients.
- *  Multiple versions of dot product are supplied to calculate 1-6 output
- *  vectors in one pass.
- *  Base GF multiply and divide functions can be sped up by defining
- *  GF_LARGE_TABLES at the expense of memory size.
- *
- */
-
-/**
- * Return 0 if not support, 1 otherwise.
- */
-int build_support_erasurecode();
-
-/**
- * Get the library name possibly of full path.
- */
-const char* get_library_name();
-
-/**
- * Initialize and load erasure code library, returning error message if any.
- *
- * @param err     The err message buffer.
- * @param err_len The length of the message buffer.
- */
-void load_erasurecode_lib(char* err, size_t err_len);
-
-/**
- * Initialize tables for fast Erasure Code encode and decode.
- *
- * Generates the expanded tables needed for fast encode or decode for erasure
- * codes on blocks of data.  32bytes is generated for each input coefficient.
- *
- * @param k      The number of vector sources or rows in the generator matrix
- *               for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param a      Pointer to sets of arrays of input coefficients used to encode
- *               or decode data.
- * @param gftbls Pointer to start of space for concatenated output tables
- *               generated from input coefficients.  Must be of size 32*k*rows.
- * @returns none
- */
-void h_ec_init_tables(int k, int rows, unsigned char* a, unsigned char* gftbls);
-
-/**
- * Generate or decode erasure codes on blocks of data, runs appropriate version.
- *
- * Given a list of source data blocks, generate one or multiple blocks of
- * encoded data as specified by a matrix of GF(2^8) coefficients. When given a
- * suitable set of coefficients, this function will perform the fast generation
- * or decoding of Reed-Solomon type erasure codes.
- *
- * This function determines what instruction sets are enabled and
- * selects the appropriate version at runtime.
- *
- * @param len    Length of each block of data (vector) of source or dest data.
- * @param k      The number of vector sources or rows in the generator matrix
- *        for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param gftbls Pointer to array of input tables generated from coding
- *        coefficients in ec_init_tables(). Must be of size 32*k*rows
- * @param data   Array of pointers to source input buffers.
- * @param coding Array of pointers to coded output buffers.
- * @returns none
- */
-void h_ec_encode_data(int len, int k, int rows, unsigned char *gftbls,
-                                 unsigned char **data, unsigned char **coding);
-
-/**
- * @brief Generate update for encode or decode of erasure codes from single
- *        source, runs appropriate version.
- *
- * Given one source data block, update one or multiple blocks of encoded data as
- * specified by a matrix of GF(2^8) coefficients. When given a suitable set of
- * coefficients, this function will perform the fast generation or decoding of
- * Reed-Solomon type erasure codes from one input source at a time.
- *
- * This function determines what instruction sets are enabled and selects the
- * appropriate version at runtime.
- *
- * @param len    Length of each block of data (vector) of source or dest data.
- * @param k      The number of vector sources or rows in the generator matrix
- *               for coding.
- * @param rows   The number of output vectors to concurrently encode/decode.
- * @param vec_i  The vector index corresponding to the single input source.
- * @param gftbls Pointer to array of input tables generated from coding
- *               coefficients in ec_init_tables(). Must be of size 32*k*rows
- * @param data   Pointer to single input source used to update output parity.
- * @param coding Array of pointers to coded output buffers.
- * @returns none
- */
-void h_ec_encode_data_update(int len, int k, int rows, int vec_i,
-           unsigned char *gftbls, unsigned char *data, unsigned char **coding);
-
-#endif //_ERASURE_CODE_H_

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
deleted file mode 100644
index 2be8328..0000000
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- *  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.
- */
-
-#ifndef _GF_UTIL_H
-#define _GF_UTIL_H
-
-/**
- *  gf_util.h
- *  Interface to functions for vector (block) multiplication in GF(2^8).
- *
- *  This file defines the interface to routines used in fast RAID rebuild and
- *  erasure codes.
- */
-
-
-/**
- * Single element GF(2^8) multiply.
- *
- * @param a  Multiplicand a
- * @param b  Multiplicand b
- * @returns  Product of a and b in GF(2^8)
- */
-unsigned char h_gf_mul(unsigned char a, unsigned char b);
-
-/**
- * Single element GF(2^8) inverse.
- *
- * @param a  Input element
- * @returns  Field element b such that a x b = {1}
- */
-unsigned char h_gf_inv(unsigned char a);
-
-/**
- * Generate a matrix of coefficients to be used for encoding.
- *
- * Vandermonde matrix example of encoding coefficients where high portion of
- * matrix is identity matrix I and lower portion is constructed as 2^{i*(j-k+1)}
- * i:{0,k-1} j:{k,m-1}. Commonly used method for choosing coefficients in
- * erasure encoding but does not guarantee invertable for every sub matrix.  For
- * large k it is possible to find cases where the decode matrix chosen from
- * sources and parity not in erasure are not invertable. Users may want to
- * adjust for k > 5.
- *
- * @param a  [mxk] array to hold coefficients
- * @param m  number of rows in matrix corresponding to srcs + parity.
- * @param k  number of columns in matrix corresponding to srcs.
- * @returns  none
- */
-void h_gf_gen_rs_matrix(unsigned char *a, int m, int k);
-
-/**
- * Generate a Cauchy matrix of coefficients to be used for encoding.
- *
- * Cauchy matrix example of encoding coefficients where high portion of matrix
- * is identity matrix I and lower portion is constructed as 1/(i + j) | i != j,
- * i:{0,k-1} j:{k,m-1}.  Any sub-matrix of a Cauchy matrix should be invertable.
- *
- * @param a  [mxk] array to hold coefficients
- * @param m  number of rows in matrix corresponding to srcs + parity.
- * @param k  number of columns in matrix corresponding to srcs.
- * @returns  none
- */
-void h_gf_gen_cauchy_matrix(unsigned char *a, int m, int k);
-
-/**
- * Invert a matrix in GF(2^8)
- *
- * @param in  input matrix
- * @param out output matrix such that [in] x [out] = [I] - identity matrix
- * @param n   size of matrix [nxn]
- * @returns 0 successful, other fail on singular input matrix
- */
-int h_gf_invert_matrix(unsigned char *in, unsigned char *out, const int n);
-
-/**
- * GF(2^8) vector multiply by constant, runs appropriate version.
- *
- * Does a GF(2^8) vector multiply b = Ca where a and b are arrays and C
- * is a single field element in GF(2^8). Can be used for RAID6 rebuild
- * and partial write functions. Function requires pre-calculation of a
- * 32-element constant array based on constant C. gftbl(C) = {C{00},
- * C{01}, C{02}, ... , C{0f} }, {C{00}, C{10}, C{20}, ... , C{f0} }.
- * Len and src must be aligned to 32B.
- *
- * This function determines what instruction sets are enabled
- * and selects the appropriate version at runtime.
- *
- * @param len   Length of vector in bytes. Must be aligned to 32B.
- * @param gftbl Pointer to 32-byte array of pre-calculated constants based on C.
- * @param src   Pointer to src data array. Must be aligned to 32B.
- * @param dest  Pointer to destination data array. Must be aligned to 32B.
- * @returns 0 pass, other fail
- */
-int h_gf_vect_mul(int len, unsigned char *gftbl, void *src, void *dest);
-
-
-#endif //_GF_UTIL_H

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
new file mode 100644
index 0000000..55e8efd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
@@ -0,0 +1,148 @@
+/*
+ *  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.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "isal_load.h"
+
+#ifdef UNIX
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <dlfcn.h>
+
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include <Windows.h>
+#endif
+
+IsaLibLoader* isaLoader;
+
+/**
+ *  isal_load.c
+ *  Utility of loading the ISA-L library and the required functions.
+ *  Building of this codes won't rely on any ISA-L source codes, but running
+ *  into this will rely on successfully loading of the dynamic library.
+ *
+ */
+
+static const char* load_functions() {
+#ifdef UNIX
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_mul), "gf_mul");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_inv), "gf_inv");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_gen_rs_matrix), "gf_gen_rs_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_gen_cauchy_matrix), "gf_gen_cauchy1_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_invert_matrix), "gf_invert_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->gf_vect_mul), "gf_vect_mul");
+
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_init_tables), "ec_init_tables");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_encode_data), "ec_encode_data");
+  EC_LOAD_DYNAMIC_SYMBOL((isaLoader->ec_encode_data_update), "ec_encode_data_update");
+#endif
+
+#ifdef WINDOWS
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_mul, (isaLoader->gf_mul), "gf_mul");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_inv, (isaLoader->gf_inv), "gf_inv");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_rs_matrix, (isaLoader->gf_gen_rs_matrix), "gf_gen_rs_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_gen_cauchy_matrix, (isaLoader->gf_gen_cauchy_matrix), "gf_gen_cauchy1_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_invert_matrix, (isaLoader->gf_invert_matrix), "gf_invert_matrix");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_gf_vect_mul, (isaLoader->gf_vect_mul), "gf_vect_mul");
+
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_init_tables, (isaLoader->ec_init_tables), "ec_init_tables");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data, (isaLoader->ec_encode_data), "ec_encode_data");
+  EC_LOAD_DYNAMIC_SYMBOL(__d_ec_encode_data_update, (isaLoader->ec_encode_data_update), "ec_encode_data_update");
+#endif
+
+  return NULL;
+}
+
+void load_erasurecode_lib(char* err, size_t err_len) {
+  const char* errMsg;
+
+  err[0] = '\0';
+
+  if (isaLoader != NULL) {
+    return;
+  }
+  isaLoader = calloc(1, sizeof(IsaLibLoader));
+  memset(isaLoader, 0, sizeof(IsaLibLoader));
+
+  // Load Intel ISA-L
+  #ifdef UNIX
+  isaLoader->libec = dlopen(HADOOP_ISAL_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
+  if (isaLoader->libec == NULL) {
+    snprintf(err, err_len, "Failed to load %s (%s)",
+                             HADOOP_ISAL_LIBRARY, dlerror());
+    return;
+  }
+  // Clear any existing error
+  dlerror();
+  #endif
+
+  #ifdef WINDOWS
+  isaLoader->libec = LoadLibrary(HADOOP_ISAL_LIBRARY);
+  if (isaLoader->libec == NULL) {
+    snprintf(err, err_len, "Failed to load %s", HADOOP_ISAL_LIBRARY);
+    return;
+  }
+  #endif
+
+  errMsg = load_functions(isaLoader->libec);
+  if (errMsg != NULL) {
+    snprintf(err, err_len, "Loading functions from ISA-L failed: %s", errMsg);
+  }
+}
+
+int build_support_erasurecode() {
+#ifdef HADOOP_ISAL_LIBRARY
+  return 1;
+#else
+  return 0;
+#endif
+}
+
+const char* get_library_name() {
+#ifdef UNIX
+  Dl_info dl_info;
+
+  if (isaLoader->ec_encode_data == NULL) {
+    return HADOOP_ISAL_LIBRARY;
+  }
+
+  if(dladdr(isaLoader->ec_encode_data, &dl_info)) {
+    return dl_info.dli_fname;
+  }
+#else
+  LPTSTR filename = NULL;
+
+  if (isaLoader->libec == NULL) {
+    return HADOOP_ISAL_LIBRARY;
+  }
+
+  if (GetModuleFileName(isaLoader->libec, filename, 256) > 0) {
+    return filename;
+  }
+#endif
+
+  return NULL;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
new file mode 100644
index 0000000..635706d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
@@ -0,0 +1,149 @@
+/*
+ *  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.
+ */
+
+#include <errno.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+
+#ifdef UNIX
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <dlfcn.h>
+
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include <Windows.h>
+#endif
+
+#ifndef _ISAL_LOAD_H_
+#define _ISAL_LOAD_H_
+
+
+#ifdef UNIX
+// For gf_util.h
+typedef unsigned char (*__d_gf_mul)(unsigned char, unsigned char);
+typedef unsigned char (*__d_gf_inv)(unsigned char);
+typedef void (*__d_gf_gen_rs_matrix)(unsigned char *, int, int);
+typedef void (*__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
+typedef int (*__d_gf_invert_matrix)(unsigned char *, unsigned char *, const int);
+typedef int (*__d_gf_vect_mul)(int, unsigned char *, void *, void *);
+
+// For erasure_code.h
+typedef void (*__d_ec_init_tables)(int, int, unsigned char*, unsigned char*);
+typedef void (*__d_ec_encode_data)(int, int, int, unsigned char*,
+                                          unsigned char**, unsigned char**);
+typedef void (*__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
+                                             unsigned char*, unsigned char**);
+#endif
+
+#ifdef WINDOWS
+// For erasure_code.h
+typedef unsigned char (__cdecl *__d_gf_mul)(unsigned char, unsigned char);
+typedef unsigned char (__cdecl *__d_gf_inv)(unsigned char);
+typedef void (__cdecl *__d_gf_gen_rs_matrix)(unsigned char *, int, int);
+typedef void (__cdecl *__d_gf_gen_cauchy_matrix)(unsigned char *, int, int);
+typedef int (__cdecl *__d_gf_invert_matrix)(unsigned char *,
+                                                unsigned char *, const int);
+typedef int (__cdecl *__d_gf_vect_mul)(int, unsigned char *, void *, void *);
+
+// For erasure_code.h
+typedef void (__cdecl *__d_ec_init_tables)(int, int,
+                                                unsigned char*, unsigned char*);
+typedef void (__cdecl *__d_ec_encode_data)(int, int, int, unsigned char*,
+                                             unsigned char**, unsigned char**);
+typedef void (__cdecl *__d_ec_encode_data_update)(int, int, int, int, unsigned char*,
+                                             unsigned char*, unsigned char**);
+#endif
+
+typedef struct __IsaLibLoader {
+  // The loaded library handle
+  void* libec;
+
+  __d_gf_mul gf_mul;
+  __d_gf_inv gf_inv;
+  __d_gf_gen_rs_matrix gf_gen_rs_matrix;
+  __d_gf_gen_cauchy_matrix gf_gen_cauchy_matrix;
+  __d_gf_invert_matrix gf_invert_matrix;
+  __d_gf_vect_mul gf_vect_mul;
+  __d_ec_init_tables ec_init_tables;
+  __d_ec_encode_data ec_encode_data;
+  __d_ec_encode_data_update ec_encode_data_update;
+} IsaLibLoader;
+
+extern IsaLibLoader* isaLoader;
+
+/**
+ * A helper function to dlsym a 'symbol' from a given library-handle.
+ */
+
+#ifdef UNIX
+
+static __attribute__ ((unused))
+void *myDlsym(void *handle, const char *symbol) {
+  void *func_ptr = dlsym(handle, symbol);
+  return func_ptr;
+}
+
+/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
+#define EC_LOAD_DYNAMIC_SYMBOL(func_ptr, symbol) \
+  if ((func_ptr = myDlsym(isaLoader->libec, symbol)) == NULL) { \
+    return "Failed to load symbol" symbol; \
+  }
+
+#endif
+
+#ifdef WINDOWS
+
+
+static FARPROC WINAPI myDlsym(HMODULE handle, LPCSTR symbol) {
+  FARPROC func_ptr = GetProcAddress(handle, symbol);
+  return func_ptr;
+}
+
+/* A helper macro to dlsym the requisite dynamic symbol in NON-JNI env. */
+#define EC_LOAD_DYNAMIC_SYMBOL(func_type, func_ptr, symbol) \
+  if ((func_ptr = (func_type)myDlsym(isaLoader->libec, symbol)) == NULL) { \
+    return "Failed to load symbol" symbol; \
+  }
+
+#endif
+
+/**
+ * Return 0 if not support, 1 otherwise.
+ */
+int build_support_erasurecode();
+
+/**
+ * Get the library name possibly of full path.
+ */
+const char* get_library_name();
+
+/**
+ * Initialize and load erasure code library, returning error message if any.
+ *
+ * @param err     The err message buffer.
+ * @param err_len The length of the message buffer.
+ */
+void load_erasurecode_lib(char* err, size_t err_len);
+
+#endif //_ISAL_LOAD_H_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
new file mode 100644
index 0000000..8126e9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
@@ -0,0 +1,98 @@
+/**
+ * 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.
+ */
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "isal_load.h"
+#include "erasure_code.h"
+#include "jni_common.h"
+
+void loadLib(JNIEnv *env) {
+  char errMsg[1024];
+  load_erasurecode_lib(errMsg, sizeof(errMsg));
+  if (strlen(errMsg) > 0) {
+    THROW(env, "java/lang/UnsatisfiedLinkError", errMsg);
+  }
+}
+
+void setCoder(JNIEnv* env, jobject thiz, IsalCoder* pCoder) {
+  jclass clazz = (*env)->GetObjectClass(env, thiz);
+  jfieldID __coderState = (*env)->GetFieldID(env, clazz, "__native_coder", "J");
+  (*env)->SetLongField(env, thiz, __coderState, (jlong) pCoder);
+}
+
+IsalCoder* getCoder(JNIEnv* env, jobject thiz) {
+  jclass clazz = (*env)->GetObjectClass(env, thiz);
+
+  jfieldID __verbose = (*env)->GetFieldID(env, clazz, "__native_verbose", "J");
+  int verbose = (int)(*env)->GetIntField(env, thiz, __verbose);
+
+  jfieldID __coderState = (*env)->GetFieldID(env, clazz, "__native_coder", "J");
+  IsalCoder* pCoder = (IsalCoder*)(*env)->GetLongField(env,
+                                                       thiz, __coderState);
+  pCoder->verbose = verbose;
+
+  return pCoder;
+}
+
+void getInputs(JNIEnv *env, jobjectArray inputs, jintArray inputOffsets,
+                              unsigned char** destInputs, int num) {
+  int numInputs = (*env)->GetArrayLength(env, inputs);
+  int* tmpInputOffsets;
+  int i;
+  jobject byteBuffer;
+
+  if (numInputs != num) {
+    THROW(env, "java/lang/InternalError", "Invalid inputs");
+  }
+
+  tmpInputOffsets = (int*)(*env)->GetIntArrayElements(env,
+                                                      inputOffsets, NULL);
+  for (i = 0; i < numInputs; i++) {
+    byteBuffer = (*env)->GetObjectArrayElement(env, inputs, i);
+    if (byteBuffer != NULL) {
+      destInputs[i] = (unsigned char *)((*env)->GetDirectBufferAddress(env,
+                                                                byteBuffer));
+      destInputs[i] += tmpInputOffsets[i];
+    } else {
+      destInputs[i] = NULL;
+    }
+  }
+}
+
+void getOutputs(JNIEnv *env, jobjectArray outputs, jintArray outputOffsets,
+                              unsigned char** destOutputs, int num) {
+  int numOutputs = (*env)->GetArrayLength(env, outputs);
+  int i, *tmpOutputOffsets;
+  jobject byteBuffer;
+
+  if (numOutputs != num) {
+    THROW(env, "java/lang/InternalError", "Invalid outputs");
+  }
+
+  tmpOutputOffsets = (int*)(*env)->GetIntArrayElements(env,
+                                                          outputOffsets, NULL);
+  for (i = 0; i < numOutputs; i++) {
+    byteBuffer = (*env)->GetObjectArrayElement(env, outputs, i);
+    destOutputs[i] = (unsigned char *)((*env)->GetDirectBufferAddress(env,
+                                                                  byteBuffer));
+    destOutputs[i] += tmpOutputOffsets[i];
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
new file mode 100644
index 0000000..a9f62a0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
@@ -0,0 +1,42 @@
+/**
+ * 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.
+ */
+
+#ifndef _JNI_CODER_COMMON_H_
+#define _JNI_CODER_COMMON_H_
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include <jni.h>
+
+#include "erasure_coder.h"
+
+void loadLib(JNIEnv *env);
+
+void setCoder(JNIEnv* env, jobject thiz, IsalCoder* coder);
+
+IsalCoder* getCoder(JNIEnv* env, jobject thiz);
+
+void getInputs(JNIEnv *env, jobjectArray inputs, jintArray inputOffsets,
+                              unsigned char** destInputs, int num);
+
+void getOutputs(JNIEnv *env, jobjectArray outputs, jintArray outputOffsets,
+                              unsigned char** destOutputs, int num);
+
+#endif //_JNI_CODER_COMMON_H_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
new file mode 100644
index 0000000..eb09e73
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
@@ -0,0 +1,45 @@
+/**
+ * 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.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_ErasureCodeNative.h"
+
+#ifdef UNIX
+#include "config.h"
+#endif
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
+(JNIEnv *env, jclass myclass) {
+  loadLib(env);
+}
+
+JNIEXPORT jstring JNICALL
+Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
+(JNIEnv *env, jclass myclass) {
+  char* libName = get_library_name();
+  if (libName == NULL) {
+    libName = "Unavailable";
+  }
+  return (*env)->NewStringUTF(env, libName);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
new file mode 100644
index 0000000..eb4b903
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
@@ -0,0 +1,72 @@
+/**
+ * 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.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h"
+
+typedef struct _RSDecoder {
+  IsalDecoder decoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[MMAX];
+} RSDecoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_initImpl(
+JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  RSDecoder* rsDecoder = (RSDecoder*)malloc(sizeof(RSDecoder));
+  memset(rsDecoder, 0, sizeof(*rsDecoder));
+  initDecoder(&rsDecoder->decoder, (int)numDataUnits, (int)numParityUnits);
+
+  setCoder(env, thiz, &rsDecoder->decoder.coder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_decodeImpl(
+JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+jint dataLen, jintArray erasedIndexes, jobjectArray outputs,
+jintArray outputOffsets) {
+  RSDecoder* rsDecoder = (RSDecoder*)getCoder(env, thiz);
+
+  int numDataUnits = rsDecoder->decoder.coder.numDataUnits;
+  int numParityUnits = rsDecoder->decoder.coder.numParityUnits;
+  int chunkSize = (int)dataLen;
+
+  int* tmpErasedIndexes = (int*)(*env)->GetIntArrayElements(env,
+                                                        erasedIndexes, NULL);
+  int numErased = (*env)->GetArrayLength(env, erasedIndexes);
+  getInputs(env, inputs, inputOffsets, rsDecoder->inputs,
+                                               numDataUnits + numParityUnits);
+  getOutputs(env, outputs, outputOffsets, rsDecoder->outputs, numErased);
+
+  decode(&rsDecoder->decoder, rsDecoder->inputs, tmpErasedIndexes,
+                           numErased, rsDecoder->outputs, chunkSize);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_destroyImpl(
+JNIEnv *env, jobject thiz) {
+  RSDecoder* rsDecoder = (RSDecoder*)getCoder(env, thiz);
+  free(rsDecoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
new file mode 100644
index 0000000..6c477ed
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
@@ -0,0 +1,66 @@
+/**
+ * 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.
+ */
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "org_apache_hadoop.h"
+#include "erasure_code.h"
+#include "gf_util.h"
+#include "jni_common.h"
+#include "org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h"
+
+typedef struct _RSEncoder {
+  IsalEncoder encoder;
+  unsigned char* inputs[MMAX];
+  unsigned char* outputs[MMAX];
+} RSEncoder;
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_initImpl(
+JNIEnv *env, jobject thiz, jint numDataUnits, jint numParityUnits) {
+  RSEncoder* rsEncoder = (RSEncoder*)malloc(sizeof(RSEncoder));
+  memset(rsEncoder, 0, sizeof(*rsEncoder));
+  initEncoder(&rsEncoder->encoder, (int)numDataUnits, (int)numParityUnits);
+
+  setCoder(env, thiz, &rsEncoder->encoder.coder);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_encodeImpl(
+JNIEnv *env, jobject thiz, jobjectArray inputs, jintArray inputOffsets,
+jint dataLen, jobjectArray outputs, jintArray outputOffsets) {
+  RSEncoder* rsEncoder = (RSEncoder*)getCoder(env, thiz);
+
+  int numDataUnits = rsEncoder->encoder.coder.numDataUnits;
+  int numParityUnits = rsEncoder->encoder.coder.numParityUnits;
+  int chunkSize = (int)dataLen;
+
+  getInputs(env, inputs, inputOffsets, rsEncoder->inputs, numDataUnits);
+  getOutputs(env, outputs, outputOffsets, rsEncoder->outputs, numParityUnits);
+
+  encode(&rsEncoder->encoder, rsEncoder->inputs, rsEncoder->outputs, chunkSize);
+}
+
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_destroyImpl(
+JNIEnv *env, jobject thiz) {
+  RSEncoder* rsEncoder = (RSEncoder*)getCoder(env, thiz);
+  free(rsEncoder);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
new file mode 100644
index 0000000..d8ff3a0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
@@ -0,0 +1,29 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_ErasureCodeNative */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
+#define _Included_org_apache_hadoop_io_erasurecode_ErasureCodeNative
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
+ * Method:    loadLibrary
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_loadLibrary
+  (JNIEnv *, jclass);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_ErasureCodeNative
+ * Method:    getLibraryName
+ * Signature: ()Ljava/lang/String;
+ */
+JNIEXPORT jstring JNICALL Java_org_apache_hadoop_io_erasurecode_ErasureCodeNative_getLibraryName
+  (JNIEnv *, jclass);
+
+#ifdef __cplusplus
+}
+#endif
+#endif


[15/20] hadoop git commit: HDFS-9942. Add an HTrace span when refreshing the groups for a username (cmccabe)

Posted by ar...@apache.org.
HDFS-9942. Add an HTrace span when refreshing the groups for a username (cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 6e9a582eb1ab040f532fbe95694f5272cda34188
Parents: 7542996
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Mar 11 12:26:40 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Mar 11 12:26:40 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/security/Groups.java    | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e9a582e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
index 9fd39b0..11d8f2a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
@@ -27,6 +27,8 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Ticker;
 import com.google.common.cache.CacheBuilder;
@@ -217,7 +219,20 @@ public class Groups {
      */
     @Override
     public List<String> load(String user) throws Exception {
-      List<String> groups = fetchGroupList(user);
+      TraceScope scope = null;
+      Tracer tracer = Tracer.curThreadTracer();
+      if (tracer != null) {
+        scope = tracer.newScope("Groups#fetchGroupList");
+        scope.addKVAnnotation("user", user);
+      }
+      List<String> groups = null;
+      try {
+        groups = fetchGroupList(user);
+      } finally {
+        if (scope != null) {
+          scope.close();
+        }
+      }
 
       if (groups.isEmpty()) {
         if (isNegativeCacheEnabled()) {


[03/20] hadoop git commit: HADOOP-12906. AuthenticatedURL should convert a 404/Not Found into an FileNotFoundException. (Steve Loughran via gtcarrera9)

Posted by ar...@apache.org.
HADOOP-12906. AuthenticatedURL should convert a 404/Not Found into an FileNotFoundException. (Steve Loughran via gtcarrera9)


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

Branch: refs/heads/HDFS-1312
Commit: 9a79b738c582bd84727831987b845535625d75fe
Parents: d49cfb3
Author: Li Lu <gt...@apache.org>
Authored: Thu Mar 10 11:38:31 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Thu Mar 10 11:38:31 2016 -0800

----------------------------------------------------------------------
 .../security/authentication/client/AuthenticatedURL.java  | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9a79b738/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
index c50a516..f87d9d8 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
@@ -15,6 +15,7 @@ package org.apache.hadoop.security.authentication.client;
 
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -269,10 +270,15 @@ public class AuthenticatedURL {
           }
         }
       }
+    } else if (respCode == HttpURLConnection.HTTP_NOT_FOUND) {
+      token.set(null);
+      throw new FileNotFoundException(conn.getURL().toString());
     } else {
       token.set(null);
-      throw new AuthenticationException("Authentication failed, status: " + conn.getResponseCode() +
-                                        ", message: " + conn.getResponseMessage());
+      throw new AuthenticationException("Authentication failed" +
+          ", URL: " + conn.getURL() +
+          ", status: " + conn.getResponseCode() +
+          ", message: " + conn.getResponseMessage());
     }
   }
 


[20/20] hadoop git commit: Merge branch 'trunk' into HDFS-1312

Posted by ar...@apache.org.
Merge branch 'trunk' into HDFS-1312


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

Branch: refs/heads/HDFS-1312
Commit: 11045091b761ffbddb8a9eefc1b53e1f1575a8dc
Parents: 4f98af4 5644137
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Mar 14 12:59:53 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Mar 14 12:59:53 2016 -0700

----------------------------------------------------------------------
 .../authentication/client/AuthenticatedURL.java |  10 +-
 .../hadoop-common/src/CMakeLists.txt            |  22 +-
 .../java/org/apache/hadoop/security/Groups.java |  17 +-
 .../authorize/ServiceAuthorizationManager.java  |  12 +-
 .../src/main/native/native.vcxproj              |  11 +-
 .../io/erasurecode/coder/erasure_code_native.c  |  49 ---
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 --
 .../src/org/apache/hadoop/io/erasurecode/dump.c | 100 ++++++
 .../src/org/apache/hadoop/io/erasurecode/dump.h |  40 +++
 .../apache/hadoop/io/erasurecode/erasure_code.c | 239 +--------------
 .../apache/hadoop/io/erasurecode/erasure_code.h | 107 +++++++
 .../hadoop/io/erasurecode/erasure_coder.c       | 229 ++++++++++++++
 .../hadoop/io/erasurecode/erasure_coder.h       |  88 ++++++
 .../org/apache/hadoop/io/erasurecode/gf_util.c  |  54 ++++
 .../org/apache/hadoop/io/erasurecode/gf_util.h  | 111 +++++++
 .../io/erasurecode/include/erasure_code.h       | 125 --------
 .../hadoop/io/erasurecode/include/gf_util.h     | 111 -------
 .../apache/hadoop/io/erasurecode/isal_load.c    | 148 +++++++++
 .../apache/hadoop/io/erasurecode/isal_load.h    | 149 +++++++++
 .../apache/hadoop/io/erasurecode/jni_common.c   |  98 ++++++
 .../apache/hadoop/io/erasurecode/jni_common.h   |  42 +++
 .../io/erasurecode/jni_erasure_code_native.c    |  45 +++
 .../hadoop/io/erasurecode/jni_rs_decoder.c      |  72 +++++
 .../hadoop/io/erasurecode/jni_rs_encoder.c      |  66 ++++
 ...he_hadoop_io_erasurecode_ErasureCodeNative.h |  29 ++
 ...io_erasurecode_rawcoder_NativeRSRawDecoder.h |  37 +++
 ...io_erasurecode_rawcoder_NativeRSRawEncoder.h |  37 +++
 .../hadoop/io/erasurecode/erasure_code_test.c   | 307 ++++---------------
 hadoop-dist/pom.xml                             |  24 +-
 .../server/blockmanagement/BlockManager.java    |   3 +-
 .../BlockUnderConstructionFeature.java          |  45 +--
 .../server/blockmanagement/DatanodeManager.java |  44 ++-
 .../blockmanagement/DecommissionManager.java    |   8 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   1 +
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  18 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   | 103 ++++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  32 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   2 +-
 .../OfflineImageReconstructor.java              |  11 +
 .../OfflineImageViewerPB.java                   |   2 +-
 .../src/site/markdown/HdfsImageViewer.md        |  15 +-
 .../TestComputeInvalidateWork.java              |   2 +-
 .../namenode/TestNameNodeReconfigure.java       | 126 ++++++++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 134 ++++----
 .../TestOfflineImageViewer.java                 |  34 ++
 .../hadoop/conf/TestNoDefaultsJobConf.java      |  30 +-
 .../apache/hadoop/mapred/HadoopTestCase.java    |  17 +-
 .../hadoop/mapred/NotificationTestCase.java     |  15 +-
 .../hadoop/mapred/TestFileOutputFormat.java     |   4 +
 .../apache/hadoop/mapred/TestTaskCommit.java    |  31 +-
 .../mapred/jobcontrol/TestLocalJobControl.java  |   3 +
 .../hadoop/mapred/lib/TestChainMapReduce.java   |  20 +-
 .../mapred/lib/TestKeyFieldBasedComparator.java |   2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   |  17 +-
 .../hadoop/mapred/lib/TestMultipleOutputs.java  |  26 +-
 .../mapred/lib/TestMultithreadedMapRunner.java  |  21 +-
 .../org/apache/hadoop/mapreduce/TestChild.java  |  10 +-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |   6 +-
 .../hadoop/mapreduce/TestTaskContext.java       |   2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |   9 +
 .../mapreduce/lib/chain/TestMapReduceChain.java |   6 +
 .../lib/chain/TestSingleElementChain.java       |   4 +
 .../lib/db/TestDataDrivenDBInputFormat.java     |  40 ++-
 .../mapreduce/lib/input/TestMultipleInputs.java |   6 +-
 .../lib/jobcontrol/TestMapReduceJobControl.java |   8 +-
 .../lib/map/TestMultithreadedMapper.java        |  12 +-
 .../lib/output/TestJobOutputCommitter.java      |  16 +-
 .../lib/output/TestMRMultipleOutputs.java       |  17 +-
 .../TestMRKeyFieldBasedComparator.java          |  20 +-
 .../hadoop/examples/terasort/TestTeraSort.java  |  11 +-
 hadoop-project/pom.xml                          |  13 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../pom.xml                                     |  21 ++
 .../distributedshell/ApplicationMaster.java     |  68 ++--
 .../DistributedShellTimelinePlugin.java         |  79 +++++
 .../distributedshell/package-info.java          |  19 ++
 .../distributedshell/TestDistributedShell.java  | 120 +++++++-
 .../hadoop/yarn/client/api/TimelineClient.java  |   4 +-
 .../api/impl/FileSystemTimelineWriter.java      |  51 +--
 .../client/api/impl/TimelineClientImpl.java     |  13 +
 .../yarn/client/api/impl/TimelineWriter.java    |  40 ++-
 .../yarn/util/timeline/TimelineUtils.java       |  35 +++
 .../timeline/webapp/TimelineWebServices.java    |  12 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     |  10 +-
 .../yarn/server/timeline/TimelineVersion.java   |  31 ++
 .../server/timeline/TimelineVersionWatcher.java |  47 +++
 .../pom.xml                                     |  16 +
 .../yarn/server/timeline/EntityCacheItem.java   |  40 +--
 .../timeline/EntityGroupFSTimelineStore.java    | 204 +++++++++---
 .../hadoop/yarn/server/timeline/LogInfo.java    |  11 +-
 .../server/timeline/PluginStoreTestUtils.java   |  51 ++-
 .../TestEntityGroupFSTimelineStore.java         |   8 +-
 .../TestOverrideTimelineStoreYarnClient.java    |  56 ++++
 93 files changed, 3146 insertions(+), 1150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11045091/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------


[07/20] hadoop git commit: HDFS-9933. ReverseXML should be capitalized in oiv usage message (cmccabe)

Posted by ar...@apache.org.
HDFS-9933. ReverseXML should be capitalized in oiv usage message (cmccabe)


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

Branch: refs/heads/HDFS-1312
Commit: 79961ecea888e0ee85b7a75e239bb6bb3335eb17
Parents: bd49354
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Thu Mar 10 13:43:27 2016 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Thu Mar 10 13:43:27 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79961ece/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
index e184804..b514b3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
@@ -60,7 +60,7 @@ public class OfflineImageViewerPB {
       + "  * XML: This processor creates an XML document with all elements of\n"
       + "    the fsimage enumerated, suitable for further analysis by XML\n"
       + "    tools.\n"
-      + "  * reverseXML: This processor takes an XML file and creates a\n"
+      + "  * ReverseXML: This processor takes an XML file and creates a\n"
       + "    binary fsimage containing the same elements.\n"
       + "  * FileDistribution: This processor analyzes the file size\n"
       + "    distribution in the image.\n"


[09/20] hadoop git commit: HDFS-1477. Support reconfiguring dfs.heartbeat.interval and dfs.namenode.heartbeat.recheck-interval without NN restart. (Contributed by Xiaobing Zhou)

Posted by ar...@apache.org.
HDFS-1477. Support reconfiguring dfs.heartbeat.interval and dfs.namenode.heartbeat.recheck-interval without NN restart. (Contributed by Xiaobing Zhou)


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

Branch: refs/heads/HDFS-1312
Commit: e01c6ea688e62f25c4310e771a0cd85b53a5fb87
Parents: adf1cdf
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Mar 10 19:03:55 2016 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Mar 10 19:03:55 2016 -0800

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    |   3 +-
 .../server/blockmanagement/DatanodeManager.java |  44 +++++-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   1 +
 .../hadoop/hdfs/server/namenode/NameNode.java   | 103 ++++++++++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  28 ++--
 .../hdfs/server/namenode/NamenodeFsck.java      |   2 +-
 .../TestComputeInvalidateWork.java              |   2 +-
 .../namenode/TestNameNodeReconfigure.java       | 126 +++++++++++++++++
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 134 +++++++++++--------
 9 files changed, 357 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index f12ea1b..6ed102c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -332,7 +332,8 @@ public class BlockManager implements BlockStatsMXBean {
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY,
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L;
     invalidateBlocks = new InvalidateBlocks(
-        datanodeManager.blockInvalidateLimit, startupDelayBlockDeletionInMs);
+        datanodeManager.getBlockInvalidateLimit(),
+        startupDelayBlockDeletionInMs);
 
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 3072fc0..53c7c16 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.net.InetAddresses;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -70,6 +71,8 @@ public class DatanodeManager {
   private final HeartbeatManager heartbeatManager;
   private final FSClusterStats fsClusterStats;
 
+  private volatile long heartbeatIntervalSeconds;
+  private volatile int heartbeatRecheckInterval;
   /**
    * Stores the datanode -> block map.  
    * <p>
@@ -113,7 +116,7 @@ public class DatanodeManager {
   /** The period to wait for datanode heartbeat.*/
   private long heartbeatExpireInterval;
   /** Ask Datanode only up to this many blocks to delete. */
-  final int blockInvalidateLimit;
+  private volatile int blockInvalidateLimit;
 
   /** The interval for judging stale DataNodes for read/write */
   private final long staleInterval;
@@ -227,10 +230,10 @@ public class DatanodeManager {
       dnsToSwitchMapping.resolve(locations);
     }
 
-    final long heartbeatIntervalSeconds = conf.getLong(
+    heartbeatIntervalSeconds = conf.getLong(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT);
-    final int heartbeatRecheckInterval = conf.getInt(
+    heartbeatRecheckInterval = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 
         DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes
     this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval
@@ -348,6 +351,10 @@ public class DatanodeManager {
     return fsClusterStats;
   }
 
+  int getBlockInvalidateLimit() {
+    return blockInvalidateLimit;
+  }
+
   /** @return the datanode statistics. */
   public DatanodeStatistics getDatanodeStatistics() {
     return heartbeatManager;
@@ -1103,6 +1110,14 @@ public class DatanodeManager {
     return staleInterval;
   }
 
+  public long getHeartbeatInterval() {
+    return this.heartbeatIntervalSeconds;
+  }
+
+  public long getHeartbeatRecheckInterval() {
+    return this.heartbeatRecheckInterval;
+  }
+
   /**
    * Set the number of current stale DataNodes. The HeartbeatManager got this
    * number based on DataNodes' heartbeats.
@@ -1667,5 +1682,28 @@ public class DatanodeManager {
       }
     };
   }
+
+  public void setHeartbeatInterval(long intervalSeconds) {
+    setHeartbeatInterval(intervalSeconds,
+        this.heartbeatRecheckInterval);
+  }
+
+  public void setHeartbeatRecheckInterval(int recheckInterval) {
+    setHeartbeatInterval(this.heartbeatIntervalSeconds,
+        recheckInterval);
+  }
+
+  /**
+   * Set parameters derived from heartbeat interval.
+   */
+  private void setHeartbeatInterval(long intervalSeconds,
+      int recheckInterval) {
+    this.heartbeatIntervalSeconds = intervalSeconds;
+    this.heartbeatRecheckInterval = recheckInterval;
+    this.heartbeatExpireInterval = 2L * recheckInterval + 10 * 1000
+        * intervalSeconds;
+    this.blockInvalidateLimit = Math.max(20 * (int) (intervalSeconds),
+        DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT);
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 2362610..989afbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -2995,6 +2995,7 @@ public class DataNode extends ReconfigurableBase
   @Override // ClientDatanodeProtocol & ReconfigurationProtocol
   public List<String> listReconfigurableProperties()
       throws IOException {
+    checkSuperuserPrivilege();
     return RECONFIGURABLE_PROPERTIES;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index e8900ee..148626b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -21,11 +21,14 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurableBase;
+import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -41,6 +44,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -93,6 +97,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -141,6 +146,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SUPPORT_ALLOW_FO
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
 
@@ -182,7 +191,8 @@ import static org.apache.hadoop.util.ToolRunner.confirmPrompt;
  * NameNode state, for example partial blocksMap etc.
  **********************************************************/
 @InterfaceAudience.Private
-public class NameNode implements NameNodeStatusMXBean {
+public class NameNode extends ReconfigurableBase implements
+    NameNodeStatusMXBean {
   static{
     HdfsConfiguration.init();
   }
@@ -260,7 +270,12 @@ public class NameNode implements NameNodeStatusMXBean {
   public static final String[] NAMESERVICE_SPECIFIC_KEYS = {
     DFS_HA_AUTO_FAILOVER_ENABLED_KEY
   };
-  
+
+  /** A list of property that are reconfigurable at runtime. */
+  static final List<String> RECONFIGURABLE_PROPERTIES = Collections
+      .unmodifiableList(Arrays.asList(DFS_HEARTBEAT_INTERVAL_KEY,
+          DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY));
+
   private static final String USAGE = "Usage: hdfs namenode ["
       + StartupOption.BACKUP.getName() + "] | \n\t["
       + StartupOption.CHECKPOINT.getName() + "] | \n\t["
@@ -329,7 +344,6 @@ public class NameNode implements NameNodeStatusMXBean {
       LogFactory.getLog("NameNodeMetricsLog");
 
   protected FSNamesystem namesystem; 
-  protected final Configuration conf;
   protected final NamenodeRole role;
   private volatile HAState state;
   private final boolean haEnabled;
@@ -864,12 +878,12 @@ public class NameNode implements NameNodeStatusMXBean {
 
   protected NameNode(Configuration conf, NamenodeRole role)
       throws IOException {
+    super(conf);
     this.tracer = new Tracer.Builder("NameNode").
         conf(TraceUtils.wrapHadoopConf(NAMENODE_HTRACE_PREFIX, conf)).
         build();
     this.tracerConfigurationManager =
         new TracerConfigurationManager(NAMENODE_HTRACE_PREFIX, conf);
-    this.conf = conf;
     this.role = role;
     setClientNamenodeAddress(conf);
     String nsId = getNameServiceId(conf);
@@ -880,7 +894,7 @@ public class NameNode implements NameNodeStatusMXBean {
     this.haContext = createHAContext();
     try {
       initializeGenericKeys(conf, nsId, namenodeId);
-      initialize(conf);
+      initialize(getConf());
       try {
         haContext.writeLock();
         state.prepareToEnterState(haContext);
@@ -1804,7 +1818,7 @@ public class NameNode implements NameNodeStatusMXBean {
     public void startActiveServices() throws IOException {
       try {
         namesystem.startActiveServices();
-        startTrashEmptier(conf);
+        startTrashEmptier(getConf());
       } catch (Throwable t) {
         doImmediateShutdown(t);
       }
@@ -1825,7 +1839,7 @@ public class NameNode implements NameNodeStatusMXBean {
     @Override
     public void startStandbyServices() throws IOException {
       try {
-        namesystem.startStandbyServices(conf);
+        namesystem.startStandbyServices(getConf());
       } catch (Throwable t) {
         doImmediateShutdown(t);
       }
@@ -1902,8 +1916,8 @@ public class NameNode implements NameNodeStatusMXBean {
    */
   void checkHaStateChange(StateChangeRequestInfo req)
       throws AccessControlException {
-    boolean autoHaEnabled = conf.getBoolean(DFS_HA_AUTO_FAILOVER_ENABLED_KEY,
-        DFS_HA_AUTO_FAILOVER_ENABLED_DEFAULT);
+    boolean autoHaEnabled = getConf().getBoolean(
+        DFS_HA_AUTO_FAILOVER_ENABLED_KEY, DFS_HA_AUTO_FAILOVER_ENABLED_DEFAULT);
     switch (req.getSource()) {
     case REQUEST_BY_USER:
       if (autoHaEnabled) {
@@ -1930,4 +1944,75 @@ public class NameNode implements NameNodeStatusMXBean {
       break;
     }
   }
+
+  /*
+   * {@inheritDoc}
+   * */
+  @Override // ReconfigurableBase
+  public Collection<String> getReconfigurableProperties() {
+    return RECONFIGURABLE_PROPERTIES;
+  }
+
+  /*
+   * {@inheritDoc}
+   * */
+  @Override // ReconfigurableBase
+  protected String reconfigurePropertyImpl(String property, String newVal)
+      throws ReconfigurationException {
+    final DatanodeManager datanodeManager = namesystem.getBlockManager()
+        .getDatanodeManager();
+
+    switch (property) {
+    case DFS_HEARTBEAT_INTERVAL_KEY:
+      namesystem.writeLock();
+      try {
+        if (newVal == null) {
+          // set to default
+          datanodeManager.setHeartbeatInterval(DFS_HEARTBEAT_INTERVAL_DEFAULT);
+          return String.valueOf(DFS_HEARTBEAT_INTERVAL_DEFAULT);
+        } else {
+          datanodeManager.setHeartbeatInterval(Long.parseLong(newVal));
+          return String.valueOf(datanodeManager.getHeartbeatInterval());
+        }
+      } catch (NumberFormatException nfe) {
+        throw new ReconfigurationException(property, newVal, getConf().get(
+            property), nfe);
+      } finally {
+        namesystem.writeUnlock();
+        LOG.info("RECONFIGURE* changed heartbeatInterval to "
+            + datanodeManager.getHeartbeatInterval());
+      }
+    case DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY:
+      namesystem.writeLock();
+      try {
+        if (newVal == null) {
+          // set to default
+          datanodeManager
+              .setHeartbeatRecheckInterval(
+                  DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
+          return String
+              .valueOf(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT);
+        } else {
+          datanodeManager.setHeartbeatRecheckInterval(Integer.parseInt(newVal));
+          return String.valueOf(datanodeManager.getHeartbeatRecheckInterval());
+        }
+      } catch (NumberFormatException nfe) {
+        throw new ReconfigurationException(property, newVal, getConf().get(
+            property), nfe);
+      } finally {
+        namesystem.writeUnlock();
+        LOG.info("RECONFIGURE* changed heartbeatRecheckInterval to "
+            + datanodeManager.getHeartbeatRecheckInterval());
+      }
+    default:
+      break;
+    }
+    throw new ReconfigurationException(property, newVal, getConf()
+        .get(property));
+  }
+
+  @Override  // ReconfigurableBase
+  protected Configuration getNewConf() {
+    return new HdfsConfiguration();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 0c4a440..6dff1bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -43,7 +43,6 @@ import com.google.common.collect.Lists;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.conf.ReconfigurationTaskStatus;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
@@ -2109,7 +2108,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     namesystem.checkOperation(OperationCategory.READ); // only active
     namesystem.checkSuperuserPrivilege();
-    int maxEventsPerRPC = nn.conf.getInt(
+    int maxEventsPerRPC = nn.getConf().getInt(
         DFSConfigKeys.DFS_NAMENODE_INOTIFY_MAX_EVENTS_PER_RPC_KEY,
         DFSConfigKeys.DFS_NAMENODE_INOTIFY_MAX_EVENTS_PER_RPC_DEFAULT);
     FSEditLog log = namesystem.getFSImage().getEditLog();
@@ -2224,23 +2223,24 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ReconfigurationProtocol
-  public void startReconfiguration() {
-    throw new UnsupportedOperationException(
-        "Namenode startReconfiguration is not implemented.",
-        new ReconfigurationException());
+  public void startReconfiguration() throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    nn.startReconfigurationTask();
   }
 
   @Override // ReconfigurationProtocol
-  public ReconfigurationTaskStatus getReconfigurationStatus() {
-    throw new UnsupportedOperationException(
-        " Namenode getReconfigurationStatus is not implemented.",
-        new ReconfigurationException());
+  public ReconfigurationTaskStatus getReconfigurationStatus()
+      throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    return nn.getReconfigurationTaskStatus();
   }
 
   @Override // ReconfigurationProtocol
-  public List<String> listReconfigurableProperties() {
-    throw new UnsupportedOperationException(
-        " Namenode listReconfigurableProperties is not implemented.",
-        new ReconfigurationException());
+  public List<String> listReconfigurableProperties() throws IOException {
+    checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
+    return NameNode.RECONFIGURABLE_PROPERTIES;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 647dd83..d3be9b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -938,7 +938,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             setInetSocketAddress(targetAddr).
             setCachingStrategy(CachingStrategy.newDropBehind()).
             setClientCacheContext(dfs.getClientContext()).
-            setConfiguration(namenode.conf).
+            setConfiguration(namenode.getConf()).
             setTracer(tracer).
             setRemotePeerFactory(new RemotePeerFactory() {
               @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
index c33161f..033f4d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestComputeInvalidateWork.java
@@ -81,7 +81,7 @@ public class TestComputeInvalidateWork {
   @Test(timeout=120000)
   public void testCompInvalidate() throws Exception {
     final int blockInvalidateLimit = bm.getDatanodeManager()
-        .blockInvalidateLimit;
+        .getBlockInvalidateLimit();
     namesystem.writeLock();
     try {
       for (int i=0; i<nodes.length; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
new file mode 100644
index 0000000..abdb1ea
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java
@@ -0,0 +1,126 @@
+/**
+ * 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.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
+
+import static org.junit.Assert.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT;
+
+public class TestNameNodeReconfigure {
+
+  public static final Log LOG = LogFactory
+      .getLog(TestNameNodeReconfigure.class);
+
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setUp() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+  }
+
+  /**
+   * Test that we can modify configuration properties.
+   */
+  @Test
+  public void testReconfigure() throws ReconfigurationException {
+    final NameNode nameNode = cluster.getNameNode();
+    final DatanodeManager datanodeManager = nameNode.namesystem
+        .getBlockManager().getDatanodeManager();
+    // change properties
+    nameNode.reconfigureProperty(DFS_HEARTBEAT_INTERVAL_KEY, "" + 6);
+    nameNode.reconfigureProperty(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+        "" + (10 * 60 * 1000));
+
+    // try invalid values
+    try {
+      nameNode.reconfigureProperty(DFS_HEARTBEAT_INTERVAL_KEY, "text");
+      fail("ReconfigurationException expected");
+    } catch (ReconfigurationException expected) {
+      assertTrue(expected.getCause() instanceof NumberFormatException);
+    }
+    try {
+      nameNode.reconfigureProperty(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+          "text");
+      fail("ReconfigurationException expected");
+    } catch (ReconfigurationException expected) {
+      assertTrue(expected.getCause() instanceof NumberFormatException);
+    }
+
+    // verify change
+    assertEquals(
+        DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value",
+        6,
+        nameNode.getConf().getLong(DFS_HEARTBEAT_INTERVAL_KEY,
+            DFS_HEARTBEAT_INTERVAL_DEFAULT));
+    assertEquals(DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value", 6,
+        datanodeManager.getHeartbeatInterval());
+
+    assertEquals(
+        DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY + " has wrong value",
+        10 * 60 * 1000,
+        nameNode.getConf().getInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+            DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT));
+    assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY
+        + " has wrong value", 10 * 60 * 1000,
+        datanodeManager.getHeartbeatRecheckInterval());
+
+    // revert to defaults
+    nameNode.reconfigureProperty(DFS_HEARTBEAT_INTERVAL_KEY, null);
+    nameNode.reconfigureProperty(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
+        null);
+
+    // verify defaults
+    assertEquals(DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value", null,
+        nameNode.getConf().get(DFS_HEARTBEAT_INTERVAL_KEY));
+    assertEquals(DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value",
+        DFS_HEARTBEAT_INTERVAL_DEFAULT, datanodeManager.getHeartbeatInterval());
+
+    assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY
+        + " has wrong value", null,
+        nameNode.getConf().get(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY));
+    assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY
+        + " has wrong value", DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT,
+        datanodeManager.getHeartbeatRecheckInterval());
+  }
+
+  @After
+  public void shutDown() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e01c6ea6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index a3ed4f6..81f93aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -18,7 +18,11 @@
 package org.apache.hadoop.hdfs.tools;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.logging.Log;
@@ -31,6 +35,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,6 +47,7 @@ import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Scanner;
+import java.util.concurrent.TimeoutException;
 
 import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.anyOf;
@@ -89,12 +95,6 @@ public class TestDFSAdmin {
     namenode = cluster.getNameNode();
   }
 
-  private void startReconfiguration(String nodeType, String address,
-      final List<String> outs, final List<String> errs) throws IOException {
-    reconfigurationOutErrFormatter("startReconfiguration", nodeType,
-        address, outs, errs);
-  }
-
   private void getReconfigurableProperties(String nodeType, String address,
       final List<String> outs, final List<String> errs) throws IOException {
     reconfigurationOutErrFormatter("getReconfigurableProperties", nodeType,
@@ -151,9 +151,10 @@ public class TestDFSAdmin {
    * @param expectedSuccuss set true if the reconfiguration task should success.
    * @throws IOException
    * @throws InterruptedException
+   * @throws TimeoutException
    */
   private void testDataNodeGetReconfigurationStatus(boolean expectedSuccuss)
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, TimeoutException {
     ReconfigurationUtil ru = mock(ReconfigurationUtil.class);
     datanode.setReconfigurationUtil(ru);
 
@@ -179,21 +180,10 @@ public class TestDFSAdmin {
 
     assertThat(admin.startReconfiguration("datanode", address), is(0));
 
-    int count = 100;
     final List<String> outs = Lists.newArrayList();
     final List<String> errs = Lists.newArrayList();
-    while (count > 0) {
-      outs.clear();
-      errs.clear();
-      getReconfigurationStatus("datanode", address, outs, errs);
-      if (!outs.isEmpty() && outs.get(0).contains("finished")) {
-        break;
-      }
-      count--;
-      Thread.sleep(100);
-    }
-    LOG.info(String.format("count=%d", count));
-    assertTrue(count > 0);
+    awaitReconfigurationFinished("datanode", address, outs, errs);
+
     if (expectedSuccuss) {
       assertThat(outs.size(), is(4));
     } else {
@@ -232,59 +222,89 @@ public class TestDFSAdmin {
 
   @Test(timeout = 30000)
   public void testDataNodeGetReconfigurationStatus() throws IOException,
-      InterruptedException {
+      InterruptedException, TimeoutException {
     testDataNodeGetReconfigurationStatus(true);
     restartCluster();
     testDataNodeGetReconfigurationStatus(false);
   }
 
   @Test(timeout = 30000)
-  public void testNameNodeStartReconfiguration() throws IOException {
-    final String address = namenode.getHostAndPort();
-    final List<String> outs = Lists.newArrayList();
-    final List<String> errs = Lists.newArrayList();
-    startReconfiguration("namenode", address, outs, errs);
-    assertEquals(0, outs.size());
-    assertTrue(errs.size() > 1);
-    assertThat(
-        errs.get(0),
-        is(allOf(containsString("Namenode"), containsString("reconfiguring:"),
-            containsString("startReconfiguration"),
-            containsString("is not implemented"),
-            containsString("UnsupportedOperationException"))));
-  }
-
-  @Test(timeout = 30000)
   public void testNameNodeGetReconfigurableProperties() throws IOException {
     final String address = namenode.getHostAndPort();
     final List<String> outs = Lists.newArrayList();
     final List<String> errs = Lists.newArrayList();
     getReconfigurableProperties("namenode", address, outs, errs);
-    assertEquals(0, outs.size());
-    assertTrue(errs.size() > 1);
-    assertThat(
-        errs.get(0),
-        is(allOf(containsString("Namenode"),
-            containsString("reconfiguration:"),
-            containsString("listReconfigurableProperties"),
-            containsString("is not implemented"),
-            containsString("UnsupportedOperationException"))));
+    assertEquals(3, outs.size());
+    assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(1));
+    assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(2));
+    assertEquals(errs.size(), 0);
+  }
+
+  void awaitReconfigurationFinished(final String nodeType,
+      final String address, final List<String> outs, final List<String> errs)
+      throws TimeoutException, IOException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        outs.clear();
+        errs.clear();
+        try {
+          getReconfigurationStatus(nodeType, address, outs, errs);
+        } catch (IOException e) {
+          LOG.error(String.format(
+              "call getReconfigurationStatus on %s[%s] failed.", nodeType,
+              address), e);
+        }
+        return !outs.isEmpty() && outs.get(0).contains("finished");
+
+      }
+    }, 100, 100 * 100);
   }
 
   @Test(timeout = 30000)
-  public void testNameNodeGetReconfigurationStatus() throws IOException {
+  public void testNameNodeGetReconfigurationStatus() throws IOException,
+      InterruptedException, TimeoutException {
+    ReconfigurationUtil ru = mock(ReconfigurationUtil.class);
+    namenode.setReconfigurationUtil(ru);
     final String address = namenode.getHostAndPort();
+
+    List<ReconfigurationUtil.PropertyChange> changes =
+        new ArrayList<>();
+    changes.add(new ReconfigurationUtil.PropertyChange(
+        DFS_HEARTBEAT_INTERVAL_KEY, String.valueOf(6),
+        namenode.getConf().get(DFS_HEARTBEAT_INTERVAL_KEY)));
+    changes.add(new ReconfigurationUtil.PropertyChange(
+        "randomKey", "new123", "old456"));
+    when(ru.parseChangedProperties(any(Configuration.class),
+        any(Configuration.class))).thenReturn(changes);
+    assertThat(admin.startReconfiguration("namenode", address), is(0));
+
     final List<String> outs = Lists.newArrayList();
     final List<String> errs = Lists.newArrayList();
-    getReconfigurationStatus("namenode", address, outs, errs);
-    assertEquals(0, outs.size());
-    assertTrue(errs.size() > 1);
-    assertThat(
-        errs.get(0),
-        is(allOf(containsString("Namenode"),
-            containsString("reloading configuration:"),
-            containsString("getReconfigurationStatus"),
-            containsString("is not implemented"),
-            containsString("UnsupportedOperationException"))));
+    awaitReconfigurationFinished("namenode", address, outs, errs);
+
+    // verify change
+    assertEquals(
+        DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value",
+        6,
+        namenode
+          .getConf()
+          .getLong(DFS_HEARTBEAT_INTERVAL_KEY,
+                DFS_HEARTBEAT_INTERVAL_DEFAULT));
+    assertEquals(DFS_HEARTBEAT_INTERVAL_KEY + " has wrong value",
+        6,
+        namenode
+          .getNamesystem()
+          .getBlockManager()
+          .getDatanodeManager()
+          .getHeartbeatInterval());
+
+    int offset = 1;
+    assertThat(outs.get(offset), containsString("SUCCESS: Changed property "
+        + DFS_HEARTBEAT_INTERVAL_KEY));
+    assertThat(outs.get(offset + 1),
+        is(allOf(containsString("From:"), containsString("3"))));
+    assertThat(outs.get(offset + 2),
+        is(allOf(containsString("To:"), containsString("6"))));
   }
 }
\ No newline at end of file


[13/20] hadoop git commit: MAPREDUCE-6520. Migrate MR Client test cases part 1. Contributed by Dustin Cote.

Posted by ar...@apache.org.
MAPREDUCE-6520. Migrate MR Client test cases part 1. Contributed by Dustin Cote.


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

Branch: refs/heads/HDFS-1312
Commit: 247a7906092065289ea81139e71badcac6abef1e
Parents: 6876b9f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Mar 11 22:51:20 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Mar 11 22:51:20 2016 +0900

----------------------------------------------------------------------
 .../hadoop/conf/TestNoDefaultsJobConf.java      | 30 +++++++++++----
 .../apache/hadoop/mapred/HadoopTestCase.java    | 17 ++++-----
 .../hadoop/mapred/NotificationTestCase.java     | 15 ++++++--
 .../hadoop/mapred/TestFileOutputFormat.java     |  4 ++
 .../apache/hadoop/mapred/TestTaskCommit.java    | 31 ++++++++++-----
 .../mapred/jobcontrol/TestLocalJobControl.java  |  3 ++
 .../hadoop/mapred/lib/TestChainMapReduce.java   | 20 +++++++++-
 .../mapred/lib/TestKeyFieldBasedComparator.java |  2 +
 .../hadoop/mapred/lib/TestMultipleInputs.java   | 17 +++++----
 .../hadoop/mapred/lib/TestMultipleOutputs.java  | 26 ++++++++++++-
 .../mapred/lib/TestMultithreadedMapRunner.java  | 21 +++++++++-
 .../org/apache/hadoop/mapreduce/TestChild.java  | 10 ++++-
 .../hadoop/mapreduce/TestNoJobSetupCleanup.java |  6 ++-
 .../hadoop/mapreduce/TestTaskContext.java       |  2 +
 .../mapreduce/lib/chain/TestChainErrors.java    |  9 +++++
 .../mapreduce/lib/chain/TestMapReduceChain.java |  6 +++
 .../lib/chain/TestSingleElementChain.java       |  4 ++
 .../lib/db/TestDataDrivenDBInputFormat.java     | 40 ++++++++++++++------
 .../mapreduce/lib/input/TestMultipleInputs.java |  6 ++-
 .../lib/jobcontrol/TestMapReduceJobControl.java |  8 +++-
 .../lib/map/TestMultithreadedMapper.java        | 12 +++++-
 .../lib/output/TestJobOutputCommitter.java      | 16 ++++++--
 .../lib/output/TestMRMultipleOutputs.java       | 17 ++++++++-
 .../TestMRKeyFieldBasedComparator.java          | 20 +++++++---
 .../hadoop/examples/terasort/TestTeraSort.java  | 11 +++++-
 25 files changed, 285 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
index b47d5ef..e2d75ab 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
@@ -17,17 +17,30 @@
  */
 package org.apache.hadoop.conf;
 
-import org.junit.Assert;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-
-import java.io.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.mapred.Utils;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This testcase tests that a JobConf without default values submits jobs
@@ -40,6 +53,7 @@ public class TestNoDefaultsJobConf extends HadoopTestCase {
     super(HadoopTestCase.CLUSTER_MR, HadoopTestCase.DFS_FS, 1, 1);
   }
 
+  @Test
   public void testNoDefaults() throws Exception {
     JobConf configuration = new JobConf();
     assertTrue(configuration.get("hadoop.tmp.dir", null) != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
index 3cd0668..277c0fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.mapred;
 
-import junit.framework.TestCase;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.MRConfig;
+import org.junit.After;
+import org.junit.Before;
 
-import java.io.File;
 import java.io.IOException;
 
 /**
@@ -38,7 +37,7 @@ import java.io.IOException;
  * Job Configurations should be created using a configuration returned by the
  * 'createJobConf()' method.
  */
-public abstract class HadoopTestCase extends TestCase {
+public abstract class HadoopTestCase {
   public static final int LOCAL_MR = 1;
   public static final int CLUSTER_MR = 2;
   public static final int LOCAL_FS = 4;
@@ -140,8 +139,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public void setUp() throws Exception {
     if (localFS) {
       fileSystem = FileSystem.getLocal(new JobConf());
     }
@@ -164,7 +163,8 @@ public abstract class HadoopTestCase extends TestCase {
    *
    * @throws Exception
    */
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     try {
       if (mrCluster != null) {
         mrCluster.shutdown();
@@ -181,7 +181,6 @@ public abstract class HadoopTestCase extends TestCase {
     catch (Exception ex) {
       System.out.println(ex);
     }
-    super.tearDown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
index d2ea74e..1f657cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
@@ -34,6 +34,13 @@ import javax.servlet.ServletException;
 import java.io.IOException;
 import java.io.DataOutputStream;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+
+
 /**
  * Base class to test Job end notification in local and cluster mode.
  *
@@ -140,17 +147,19 @@ public abstract class NotificationTestCase extends HadoopTestCase {
     return conf;
   }
 
-
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     startHttpServer();
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     stopHttpServer();
     super.tearDown();
   }
 
+  @Test
   public void testMR() throws Exception {
 
     System.out.println(launchWordCount(this.createJobConf(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
index 81b53cc..3141235 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
@@ -30,12 +30,16 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Iterator;
 
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 public class TestFileOutputFormat extends HadoopTestCase {
 
   public TestFileOutputFormat() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testCustomFile() throws Exception {
     Path inDir = new Path("testing/fileoutputformat/input");
     Path outDir = new Path("testing/fileoutputformat/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
index bad06e9..bed545e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.mapred;
 
-import java.io.File;
-import java.io.IOException;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -27,9 +24,18 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.ProtocolSignature;
 import org.apache.hadoop.mapred.SortedRanges.Range;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.checkpoint.CheckpointID;
-import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID;
 import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.junit.After;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 
 
 public class TestTaskCommit extends HadoopTestCase {
@@ -80,12 +86,13 @@ public class TestTaskCommit extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
   
-  @Override
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     FileUtil.fullyDelete(new File(rootDir.toString()));
   }
-  
+
+  @Test
   public void testCommitFail() throws IOException {
     final Path inDir = new Path(rootDir, "./input");
     final Path outDir = new Path(rootDir, "./output");
@@ -199,6 +206,7 @@ public class TestTaskCommit extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testTaskCleanupDoesNotCommit() throws Exception {
     // Mimic a job with a special committer that does not cleanup
     // files when a task fails.
@@ -245,23 +253,27 @@ public class TestTaskCommit extends HadoopTestCase {
     assertTrue("Task did not succeed", umbilical.taskDone);
   }
 
+  @Test
   public void testCommitRequiredForMapTask() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     assertTrue("MapTask should need commit", testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitRequiredForReduceTask() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     assertTrue("ReduceTask should need commit", testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobSetup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobSetupTask();
     assertFalse("Job setup task should not need commit", 
         testTask.isCommitRequired());
   }
-  
+
+  @Test
   public void testCommitNotRequiredForJobCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.MAP);
     testTask.setJobCleanupTask();
@@ -269,6 +281,7 @@ public class TestTaskCommit extends HadoopTestCase {
         testTask.isCommitRequired());
   }
 
+  @Test
   public void testCommitNotRequiredForTaskCleanup() throws Exception {
     Task testTask = createDummyTask(TaskType.REDUCE);
     testTask.setTaskCleanupTask();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
index 8d35dcf..07b1306 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 /**
  * HadoopTestCase that tests the local job runner.
@@ -59,6 +61,7 @@ public class TestLocalJobControl extends HadoopTestCase {
    * object. Finally, it creates a thread to run the JobControl object and
    * monitors/reports the job states.
    */
+  @Test
   public void testLocalJobControlDataCopy() throws Exception {
 
     FileSystem fs = FileSystem.get(createJobConf());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
index 37cb91f..0933ece 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
@@ -21,12 +21,29 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestChainMapReduce extends HadoopTestCase {
 
   private static Path getFlagDir(boolean local) {
@@ -67,6 +84,7 @@ public class TestChainMapReduce extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path("testing/chain/input");
     Path outDir = new Path("testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
index 34a4d2c..35b3f24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.Utils;
 import org.junit.After;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
index e5c6d75..3a9cb9e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.mapred.lib;
 
-import java.io.IOException;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.JobConf;
@@ -30,12 +25,19 @@ import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  * @see TestDelegatingInputFormat
  */
-public class TestMultipleInputs extends TestCase {
-  
+public class TestMultipleInputs {
+
+  @Test
   public void testAddInputPathWithFormat() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -48,6 +50,7 @@ public class TestMultipleInputs extends TestCase {
        .getClass());
   }
 
+  @Test
   public void testAddInputPathWithMapper() {
     final JobConf conf = new JobConf();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
index 59c0a97..f3e5893 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
@@ -24,7 +24,23 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.DataOutputStream;
@@ -32,17 +48,23 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultipleOutputs extends HadoopTestCase {
 
   public TestMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -62,6 +84,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     return dir;
   }
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Path rootDir = getDir(ROOT_DIR);
@@ -75,6 +98,7 @@ public class TestMultipleOutputs extends HadoopTestCase {
     }
   }
 
+  @After
   public void tearDown() throws Exception {
     Path rootDir = getDir(ROOT_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
index 7e224cd..1059d29 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
@@ -22,26 +22,45 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.HadoopTestCase;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
+import org.junit.Test;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapRunner extends HadoopTestCase {
 
   public TestMultithreadedMapRunner() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
index d5afe63..338f117 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
@@ -30,6 +30,13 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Level;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 public class TestChild extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -145,7 +152,8 @@ public class TestChild extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testChild() throws Exception {
     try {
       submitAndValidateJob(createJobConf(), 1, 1, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
index 5d36c92..7520f38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Ignore;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+
 @Ignore
 public class TestNoJobSetupCleanup extends HadoopTestCase {
   private static String TEST_ROOT_DIR =
@@ -68,7 +71,8 @@ public class TestNoJobSetupCleanup extends HadoopTestCase {
         + numPartFiles, list.length == numPartFiles);
     return job;
   }
-  
+
+  @Test
   public void testNoJobSetupCleanup() throws Exception {
     try {
       Configuration conf = createJobConf();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
index bf742c4..67daaa4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
@@ -33,6 +33,8 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests context api and {@link StatusReporter#getProgress()} via 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
index 2dfcf41..46024bc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
@@ -29,6 +29,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Tests error conditions in ChainMapper/ChainReducer.
@@ -51,6 +55,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainSubmission() throws Exception {
 
     Configuration conf = createJobConf();
@@ -89,6 +94,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -114,6 +120,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testReducerFail() throws Exception {
 
     Configuration conf = createJobConf();
@@ -139,6 +146,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainMapNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";
@@ -163,6 +171,7 @@ public class TestChainErrors extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testChainReduceNoOuptut() throws Exception {
     Configuration conf = createJobConf();
     String expectedOutput = "";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
index 971ea68..aaaaf51 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
@@ -30,6 +30,11 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestMapReduceChain extends HadoopTestCase {
 
@@ -63,6 +68,7 @@ public class TestMapReduceChain extends HadoopTestCase {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
index 06cfe1c..f78ac70 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper;
 import org.apache.hadoop.mapreduce.lib.reduce.IntSumReducer;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 
@@ -42,6 +45,7 @@ public class TestSingleElementChain extends HadoopTestCase {
   }
 
   // test chain mapper and reducer by adding single mapper and reducer to chain
+  @Test
   public void testNoChain() throws Exception {
     Path inDir = new Path(localPathRoot, "testing/chain/input");
     Path outDir = new Path(localPathRoot, "testing/chain/output");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
index 37f9364..81a3249 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
@@ -18,25 +18,40 @@
 
 package org.apache.hadoop.mapreduce.lib.db;
 
-import java.sql.*;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-//import org.apache.hadoop.examples.DBCountPageView;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.db.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.lib.output.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.StringUtils;
 import org.hsqldb.server.Server;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+//import org.apache.hadoop.examples.DBCountPageView;
 
 /**
  * Test aspects of DataDrivenDBInputFormat
@@ -109,11 +124,13 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     createConnection(driverClassName, url);
   }
 
+  @Before
   public void setUp() throws Exception {
     initialize(DRIVER_CLASS, DB_URL);
     super.setUp();
   }
 
+  @After
   public void tearDown() throws Exception {
     super.tearDown();
     shutdown();
@@ -170,6 +187,7 @@ public class TestDataDrivenDBInputFormat extends HadoopTestCase {
     }
   }
 
+  @Test
   public void testDateSplits() throws Exception {
     Statement s = connection.createStatement();
     final String DATE_TABLE = "datetable";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
index c868050..632c40e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * @see TestDelegatingInputFormat
@@ -139,7 +141,7 @@ public class TestMultipleInputs extends HadoopTestCase {
     assertTrue(output.readLine().equals("e 2"));
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithFormat() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class);
@@ -152,7 +154,7 @@ public class TestMultipleInputs extends HadoopTestCase {
        .getClass());
   }
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testAddInputPathWithMapper() throws IOException {
     final Job conf = Job.getInstance();
     MultipleInputs.addInputPath(conf, new Path("/foo"), TextInputFormat.class,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
index 14c64bd..d86ddd0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 /**
  * This class performs unit test for Job/JobControl classes.
@@ -120,7 +123,8 @@ public class TestMapReduceJobControl extends HadoopTestCase {
       } catch (Exception e) {}
     }
   }
-  
+
+  @Test
   public void testJobControlWithFailJob() throws Exception {
     LOG.info("Starting testJobControlWithFailJob");
     Configuration conf = createJobConf();
@@ -144,6 +148,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControlWithKillJob() throws Exception {
     LOG.info("Starting testJobControlWithKillJob");
 
@@ -182,6 +187,7 @@ public class TestMapReduceJobControl extends HadoopTestCase {
     theControl.stop();
   }
 
+  @Test
   public void testJobControl() throws Exception {
     LOG.info("Starting testJobControl");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
index 52b0e70..5096192 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
@@ -23,23 +23,33 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMultithreadedMapper extends HadoopTestCase {
 
   public TestMultithreadedMapper() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testOKRun() throws Exception {
     run(false, false);
   }
 
+  @Test
   public void testIOExRun() throws Exception {
     run(true, false);
   }
+  @Test
   public void testRuntimeExRun() throws Exception {
     run(false, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
index 49b59ca..19b712f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
@@ -33,6 +33,11 @@ import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 /**
  * A JUnit test to test Map-Reduce job committer.
@@ -54,15 +59,15 @@ public class TestJobOutputCommitter extends HadoopTestCase {
   private FileSystem fs;
   private Configuration conf = null;
 
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     super.setUp();
     conf = createJobConf();
     fs = getFileSystem();
   }
 
-  @Override
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     fs.delete(new Path(TEST_ROOT_DIR), true);
     super.tearDown();
   }
@@ -219,6 +224,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testDefaultCleanupAndAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -238,6 +244,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * 
    * @throws Exception
    */
+  @Test
   public void testCustomAbort() throws Exception {
     // check with a successful job
     testSuccessfulJob(FileOutputCommitter.SUCCEEDED_FILE_NAME,
@@ -264,6 +271,7 @@ public class TestJobOutputCommitter extends HadoopTestCase {
    * compatibility testing.
    * @throws Exception 
    */
+  @Test
   public void testCustomCleanup() throws Exception {
     // check with a successful job
     testSuccessfulJob(CUSTOM_CLEANUP_FILE_NAME, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
index 6c432dd..babd20e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
@@ -27,23 +27,36 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.JavaSerializationComparator;
 import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestMRMultipleOutputs extends HadoopTestCase {
 
   public TestMRMultipleOutputs() throws IOException {
     super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
   }
 
+  @Test
   public void testWithoutCounters() throws Exception {
     _testMultipleOutputs(false);
     _testMOWithJavaSerialization(false);
   }
 
+  @Test
   public void testWithCounters() throws Exception {
     _testMultipleOutputs(true);
     _testMOWithJavaSerialization(true);
@@ -57,6 +70,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
   private static String TEXT = "text";
   private static String SEQUENCE = "sequence";
 
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     Configuration conf = createJobConf();
@@ -64,6 +78,7 @@ public class TestMRMultipleOutputs extends HadoopTestCase {
     fs.delete(ROOT_DIR, true);
   }
 
+  @After
   public void tearDown() throws Exception {
     Configuration conf = createJobConf();
     FileSystem fs = FileSystem.get(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
index 3a2b831..0d75d2f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.lib.partition;
 
-import java.io.*;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +30,15 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.map.InverseMapper;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 
 public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
@@ -45,8 +52,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
     conf = createJobConf();
     conf.set(MRJobConfig.MAP_OUTPUT_KEY_FIELD_SEPERATOR, " ");
   }
-  
-  private void testComparator(String keySpec, int expect) 
+
+  private void testComparator(String keySpec, int expect)
       throws Exception {
     String root = System.getProperty("test.build.data", "/tmp");
     Path inDir = new Path(root, "test_cmp/in");
@@ -93,7 +100,8 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
       reader.close();
     }
   }
-  
+
+  @Test
   public void testBasicUnixComparator() throws Exception {
     testComparator("-k1,1n", 1);
     testComparator("-k2,2n", 1);
@@ -117,7 +125,7 @@ public class TestMRKeyFieldBasedComparator extends HadoopTestCase {
   byte[] line1_bytes = line1.getBytes();
   byte[] line2_bytes = line2.getBytes();
 
-  public void testWithoutMRJob(String keySpec, int expect) throws Exception {
+  private void testWithoutMRJob(String keySpec, int expect) throws Exception {
     KeyFieldBasedComparator<Void, Void> keyFieldCmp = 
       new KeyFieldBasedComparator<Void, Void>();
     conf.set("mapreduce.partition.keycomparator.options", keySpec);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/247a7906/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index 3492089..391e482 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -27,6 +27,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
 public class TestTeraSort extends HadoopTestCase {
   private static Log LOG = LogFactory.getLog(TestTeraSort.class);
   
@@ -35,7 +41,8 @@ public class TestTeraSort extends HadoopTestCase {
     super(LOCAL_MR, LOCAL_FS, 1, 1);
   }
 
-  protected void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
     getFileSystem().delete(new Path(TEST_DIR), true);
     super.tearDown();
   }
@@ -76,6 +83,7 @@ public class TestTeraSort extends HadoopTestCase {
     assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
   }
 
+  @Test
   public void testTeraSort() throws Exception {
     // Run TeraGen to generate input for 'terasort'
     runTeraGen(createJobConf(), SORT_INPUT_PATH);
@@ -104,6 +112,7 @@ public class TestTeraSort extends HadoopTestCase {
       TERA_OUTPUT_PATH);
   }
 
+  @Test
   public void testTeraSortWithLessThanTwoArgs() throws Exception {
     String[] args = new String[1];
     assertEquals(new TeraSort().run(args), 2);


[16/20] hadoop git commit: HADOOP-11996. Improve and restructure native ISAL support (Kai Zheng via cmccabe)

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
new file mode 100644
index 0000000..40da4e1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
@@ -0,0 +1,37 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+#define _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    initImpl
+ * Signature: (II[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_initImpl
+  (JNIEnv *, jobject, jint, jint);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    decodeImpl
+ * Signature: ([Ljava/nio/ByteBuffer;[II[I[Ljava/nio/ByteBuffer;[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_decodeImpl
+  (JNIEnv *, jobject, jobjectArray, jintArray, jint, jintArray, jobjectArray, jintArray);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder
+ * Method:    destroyImpl
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder_destroyImpl
+  (JNIEnv *, jobject);
+
+#ifdef __cplusplus
+}
+#endif
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
new file mode 100644
index 0000000..db094cf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
@@ -0,0 +1,37 @@
+/* DO NOT EDIT THIS FILE - it is machine generated */
+#include <jni.h>
+/* Header for class org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder */
+
+#ifndef _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+#define _Included_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+#ifdef __cplusplus
+extern "C" {
+#endif
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    initImpl
+ * Signature: (II[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_initImpl
+  (JNIEnv *, jobject, jint, jint);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    encodeImpl
+ * Signature: ([Ljava/nio/ByteBuffer;[II[Ljava/nio/ByteBuffer;[I)V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_encodeImpl
+  (JNIEnv *, jobject, jobjectArray, jintArray, jint, jobjectArray, jintArray);
+
+/*
+ * Class:     org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder
+ * Method:    destroyImpl
+ * Signature: ()V
+ */
+JNIEXPORT void JNICALL Java_org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder_destroyImpl
+  (JNIEnv *, jobject);
+
+#ifdef __cplusplus
+}
+#endif
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/658ee95f/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c b/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
index 9817a76..331bb21 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
@@ -17,185 +17,36 @@
  */
 
 /**
- * This is a lightweight version of the same file in Intel ISA-L library to test
- * and verify the basic functions of ISA-L integration. Note it's not serving as
- * a complete ISA-L library test nor as any sample to write an erasure coder
- * using the library. A sample is to be written and provided separately.
+ * This is a sample program illustrating how to use the Intel ISA-L library.
+ * Note it's adapted from erasure_code_test.c test program, but trying to use
+ * variable names and styles we're more familiar with already similar to Java
+ * coders.
  */
 
-#include "org_apache_hadoop.h"
+#include "isal_load.h"
 #include "erasure_code.h"
 #include "gf_util.h"
+#include "erasure_coder.h"
 
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
 
-#define TEST_LEN 8192
-#define TEST_SOURCES  127
-#define MMAX TEST_SOURCES
-#define KMAX TEST_SOURCES
-#define TEST_SEED 11
-
-static void dump(unsigned char *buf, int len)
-{
-  int i;
-  for (i = 0; i < len;) {
-    printf(" %2x", 0xff & buf[i++]);
-    if (i % 32 == 0)
-      printf("\n");
-  }
-  printf("\n");
-}
-
-static void dump_matrix(unsigned char **s, int k, int m)
-{
+int main(int argc, char *argv[]) {
   int i, j;
-  for (i = 0; i < k; i++) {
-    for (j = 0; j < m; j++) {
-      printf(" %2x", s[i][j]);
-    }
-    printf("\n");
-  }
-  printf("\n");
-}
-
-static void dump_u8xu8(unsigned char *s, int k, int m)
-{
-  int i, j;
-  for (i = 0; i < k; i++) {
-    for (j = 0; j < m; j++) {
-      printf(" %2x", 0xff & s[j + (i * m)]);
-    }
-    printf("\n");
-  }
-  printf("\n");
-}
-
-// Generate Random errors
-static void gen_err_list(unsigned char *src_err_list,
-       unsigned char *src_in_err, int *pnerrs, int *pnsrcerrs, int k, int m)
-{
-  int i, err;
-  int nerrs = 0, nsrcerrs = 0;
-
-  for (i = 0, nerrs = 0, nsrcerrs = 0; i < m && nerrs < m - k; i++) {
-    err = 1 & rand();
-    src_in_err[i] = err;
-    if (err) {
-      src_err_list[nerrs++] = i;
-      if (i < k) {
-        nsrcerrs++;
-      }
-    }
-  }
-  if (nerrs == 0) { // should have at least one error
-    while ((err = (rand() % KMAX)) >= m) ;
-    src_err_list[nerrs++] = err;
-    src_in_err[err] = 1;
-    if (err < k)
-      nsrcerrs = 1;
-  }
-  *pnerrs = nerrs;
-  *pnsrcerrs = nsrcerrs;
-  return;
-}
-
-#define NO_INVERT_MATRIX -2
-// Generate decode matrix from encode matrix
-static int gf_gen_decode_matrix(unsigned char *encode_matrix,
-        unsigned char *decode_matrix,
-        unsigned char *invert_matrix,
-        unsigned int *decode_index,
-        unsigned char *src_err_list,
-        unsigned char *src_in_err,
-        int nerrs, int nsrcerrs, int k, int m)
-{
-  int i, j, p;
-  int r;
-  unsigned char *backup, *b, s;
-  int incr = 0;
-
-  b = malloc(MMAX * KMAX);
-  backup = malloc(MMAX * KMAX);
-
-  if (b == NULL || backup == NULL) {
-    printf("Test failure! Error with malloc\n");
-    free(b);
-    free(backup);
-    return -1;
-  }
-  // Construct matrix b by removing error rows
-  for (i = 0, r = 0; i < k; i++, r++) {
-    while (src_in_err[r])
-      r++;
-    for (j = 0; j < k; j++) {
-      b[k * i + j] = encode_matrix[k * r + j];
-      backup[k * i + j] = encode_matrix[k * r + j];
-    }
-    decode_index[i] = r;
-  }
-  incr = 0;
-  while (h_gf_invert_matrix(b, invert_matrix, k) < 0) {
-    if (nerrs == (m - k)) {
-      free(b);
-      free(backup);
-      printf("BAD MATRIX\n");
-      return NO_INVERT_MATRIX;
-    }
-    incr++;
-    memcpy(b, backup, MMAX * KMAX);
-    for (i = nsrcerrs; i < nerrs - nsrcerrs; i++) {
-      if (src_err_list[i] == (decode_index[k - 1] + incr)) {
-        // skip the erased parity line
-        incr++;
-        continue;
-      }
-    }
-    if (decode_index[k - 1] + incr >= m) {
-      free(b);
-      free(backup);
-      printf("BAD MATRIX\n");
-      return NO_INVERT_MATRIX;
-    }
-    decode_index[k - 1] += incr;
-    for (j = 0; j < k; j++)
-      b[k * (k - 1) + j] = encode_matrix[k * decode_index[k - 1] + j];
-
-  };
-
-  for (i = 0; i < nsrcerrs; i++) {
-    for (j = 0; j < k; j++) {
-      decode_matrix[k * i + j] = invert_matrix[k * src_err_list[i] + j];
-    }
-  }
-  /* src_err_list from encode_matrix * invert of b for parity decoding */
-  for (p = nsrcerrs; p < nerrs; p++) {
-    for (i = 0; i < k; i++) {
-      s = 0;
-      for (j = 0; j < k; j++)
-        s ^= h_gf_mul(invert_matrix[j * k + i],
-              encode_matrix[k * src_err_list[p] + j]);
-
-      decode_matrix[k * p + i] = s;
-    }
-  }
-  free(b);
-  free(backup);
-  return 0;
-}
-
-int main(int argc, char *argv[])
-{
   char err[256];
   size_t err_len = sizeof(err);
-  int re, i, j, p, m, k;
-  int nerrs, nsrcerrs;
-  unsigned int decode_index[MMAX];
-  unsigned char *temp_buffs[TEST_SOURCES], *buffs[TEST_SOURCES];
-  unsigned char *encode_matrix, *decode_matrix, *invert_matrix, *g_tbls;
-  unsigned char src_in_err[TEST_SOURCES], src_err_list[TEST_SOURCES];
-  unsigned char *recov[TEST_SOURCES];
+  int chunkSize = 1024;
+  int numDataUnits = 6;
+  int numParityUnits = 3;
+  unsigned char** dataUnits;
+  unsigned char** parityUnits;
+  IsalEncoder* pEncoder;
+  int erasedIndexes[2];
+  unsigned char* allUnits[MMAX];
+  IsalDecoder* pDecoder;
+  unsigned char* decodingOutput[2];
+  unsigned char** backupUnits;
 
   if (0 == build_support_erasurecode()) {
     printf("The native library isn't available, skipping this test\n");
@@ -209,102 +60,66 @@ int main(int argc, char *argv[])
   }
 
   printf("Performing erasure code test\n");
-  srand(TEST_SEED);
 
-  // Allocate the arrays
-  for (i = 0; i < TEST_SOURCES; i++) {
-    buffs[i] = malloc(TEST_LEN);
-  }
+  dataUnits = calloc(numDataUnits, sizeof(unsigned char*));
+  parityUnits = calloc(numParityUnits, sizeof(unsigned char*));
+  backupUnits = calloc(numParityUnits, sizeof(unsigned char*));
 
-  for (i = 0; i < TEST_SOURCES; i++) {
-    temp_buffs[i] = malloc(TEST_LEN);
+  // Allocate and generate data units
+  srand(135);
+  for (i = 0; i < numDataUnits; i++) {
+    dataUnits[i] = calloc(chunkSize, sizeof(unsigned char));
+    for (j = 0; j < chunkSize; j++) {
+      dataUnits[i][j] = rand();
+    }
   }
 
-  // Test erasure code by encode and recovery
-
-  encode_matrix = malloc(MMAX * KMAX);
-  decode_matrix = malloc(MMAX * KMAX);
-  invert_matrix = malloc(MMAX * KMAX);
-  g_tbls = malloc(KMAX * TEST_SOURCES * 32);
-  if (encode_matrix == NULL || decode_matrix == NULL
-      || invert_matrix == NULL || g_tbls == NULL) {
-    snprintf(err, err_len, "%s", "allocating test matrix buffers error");
-    return -1;
+  // Allocate and initialize parity units
+  for (i = 0; i < numParityUnits; i++) {
+    parityUnits[i] = calloc(chunkSize, sizeof(unsigned char));
+    for (j = 0; j < chunkSize; j++) {
+      parityUnits[i][j] = 0;
+    }
   }
 
-  m = 9;
-  k = 5;
-  if (m > MMAX || k > KMAX)
-    return -1;
-
-  // Make random data
-  for (i = 0; i < k; i++)
-    for (j = 0; j < TEST_LEN; j++)
-      buffs[i][j] = rand();
+  pEncoder = (IsalEncoder*)malloc(sizeof(IsalEncoder));
+  memset(pEncoder, 0, sizeof(*pEncoder));
+  initEncoder(pEncoder, numDataUnits, numParityUnits);
+  encode(pEncoder, dataUnits, parityUnits, chunkSize);
 
-  // The matrix generated by gf_gen_cauchy1_matrix
-  // is always invertable.
-  h_gf_gen_cauchy_matrix(encode_matrix, m, k);
+  pDecoder = (IsalDecoder*)malloc(sizeof(IsalDecoder));
+  memset(pDecoder, 0, sizeof(*pDecoder));
+  initDecoder(pDecoder, numDataUnits, numParityUnits);
 
-  // Generate g_tbls from encode matrix encode_matrix
-  h_ec_init_tables(k, m - k, &encode_matrix[k * k], g_tbls);
+  memcpy(allUnits, dataUnits, numDataUnits * (sizeof (unsigned char*)));
+  memcpy(allUnits + numDataUnits, parityUnits,
+                            numParityUnits * (sizeof (unsigned char*)));
 
-  // Perform matrix dot_prod for EC encoding
-  // using g_tbls from encode matrix encode_matrix
-  h_ec_encode_data(TEST_LEN, k, m - k, g_tbls, buffs, &buffs[k]);
+  erasedIndexes[0] = 1;
+  erasedIndexes[1] = 7;
 
-  // Choose random buffers to be in erasure
-  memset(src_in_err, 0, TEST_SOURCES);
-  gen_err_list(src_err_list, src_in_err, &nerrs, &nsrcerrs, k, m);
+  backupUnits[0] = allUnits[1];
+  backupUnits[1] = allUnits[7];
 
-  // Generate decode matrix
-  re = gf_gen_decode_matrix(encode_matrix, decode_matrix,
-          invert_matrix, decode_index, src_err_list, src_in_err,
-          nerrs, nsrcerrs, k, m);
-  if (re != 0) {
-    snprintf(err, err_len, "%s", "gf_gen_decode_matrix failed");
-    return -1;
-  }
-  // Pack recovery array as list of valid sources
-  // Its order must be the same as the order
-  // to generate matrix b in gf_gen_decode_matrix
-  for (i = 0; i < k; i++) {
-    recov[i] = buffs[decode_index[i]];
-  }
+  allUnits[0] = NULL; // Not to read
+  allUnits[1] = NULL;
+  allUnits[7] = NULL;
 
-  // Recover data
-  h_ec_init_tables(k, nerrs, decode_matrix, g_tbls);
-  h_ec_encode_data(TEST_LEN, k, nerrs, g_tbls, recov, &temp_buffs[k]);
-  for (i = 0; i < nerrs; i++) {
-    if (0 != memcmp(temp_buffs[k + i], buffs[src_err_list[i]], TEST_LEN)) {
-      snprintf(err, err_len, "%s", "Error recovery failed");
-      printf("Fail error recovery (%d, %d, %d)\n", m, k, nerrs);
+  decodingOutput[0] = malloc(chunkSize);
+  decodingOutput[1] = malloc(chunkSize);
 
-      printf(" - erase list = ");
-      for (j = 0; j < nerrs; j++) {
-        printf(" %d", src_err_list[j]);
-      }
-
-      printf(" - Index = ");
-      for (p = 0; p < k; p++) {
-        printf(" %d", decode_index[p]);
-      }
-
-      printf("\nencode_matrix:\n");
-      dump_u8xu8((unsigned char *) encode_matrix, m, k);
-      printf("inv b:\n");
-      dump_u8xu8((unsigned char *) invert_matrix, k, k);
-      printf("\ndecode_matrix:\n");
-      dump_u8xu8((unsigned char *) decode_matrix, m, k);
-      printf("recov %d:", src_err_list[i]);
-      dump(temp_buffs[k + i], 25);
-      printf("orig   :");
-      dump(buffs[src_err_list[i]], 25);
+  decode(pDecoder, allUnits, erasedIndexes, 2, decodingOutput, chunkSize);
 
+  for (i = 0; i < pDecoder->numErased; i++) {
+    if (0 != memcmp(decodingOutput[i], backupUnits[i], chunkSize)) {
+      fprintf(stderr, "Decoding failed\n\n");
+      dumpDecoder(pDecoder);
       return -1;
     }
   }
 
-  printf("done EC tests: Pass\n");
+  dumpDecoder(pDecoder);
+  fprintf(stdout, "Successfully done, passed!\n\n");
+
   return 0;
 }