Browse Source

Merge branch 'trunk' into HDFS-7240

Anu Engineer 9 năm trước cách đây
mục cha
commit
37e3a36a3f
100 tập tin đã thay đổi với 2928 bổ sung1475 xóa
  1. 17 5
      hadoop-common-project/hadoop-common/src/CMakeLists.txt
  2. 10 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java
  3. 6 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
  4. 6 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
  5. 4 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
  6. 147 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
  7. 0 172
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java
  8. 3 106
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  9. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  10. 16 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  11. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java
  12. 9 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  13. 9 2
      hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
  14. 100 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.c
  15. 40 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/dump.h
  16. 6 233
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.c
  17. 0 18
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h
  18. 229 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.c
  19. 88 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_coder.h
  20. 54 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.c
  21. 0 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h
  22. 148 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.c
  23. 149 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/isal_load.h
  24. 98 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.c
  25. 42 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_common.h
  26. 2 6
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c
  27. 72 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_decoder.c
  28. 66 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_rs_encoder.c
  29. 0 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h
  30. 37 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawDecoder.h
  31. 37 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_rawcoder_NativeRSRawEncoder.h
  32. 61 246
      hadoop-common-project/hadoop-common/src/main/native/src/test/org/apache/hadoop/io/erasurecode/erasure_code_test.c
  33. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java
  34. 4 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
  35. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
  36. 58 43
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
  37. 19 11
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
  38. 9 9
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
  39. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  40. 138 130
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  41. 27 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
  42. 22 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  43. 76 66
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java
  44. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java
  45. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  46. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  47. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  48. 17 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  49. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  50. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  51. 17 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  52. 103 56
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  53. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
  54. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  55. 24 24
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java
  56. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
  57. 79 79
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  58. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java
  59. 66 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
  60. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
  61. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
  62. 34 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java
  63. 6 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
  64. 22 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestNoDefaultsJobConf.java
  65. 8 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/HadoopTestCase.java
  66. 12 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java
  67. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestFileOutputFormat.java
  68. 44 45
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
  69. 22 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTaskCommit.java
  70. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/jobcontrol/TestLocalJobControl.java
  71. 19 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestChainMapReduce.java
  72. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.java
  73. 10 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleInputs.java
  74. 25 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultipleOutputs.java
  75. 20 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/lib/TestMultithreadedMapRunner.java
  76. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestChild.java
  77. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestNoJobSetupCleanup.java
  78. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestTaskContext.java
  79. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestChainErrors.java
  80. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestMapReduceChain.java
  81. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/chain/TestSingleElementChain.java
  82. 29 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/db/TestDataDrivenDBInputFormat.java
  83. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java
  84. 7 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestMapReduceJobControl.java
  85. 11 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/map/TestMultithreadedMapper.java
  86. 12 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestJobOutputCommitter.java
  87. 16 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestMRMultipleOutputs.java
  88. 14 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestMRKeyFieldBasedComparator.java
  89. 12 3
      hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java
  90. 10 1
      hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
  91. 13 0
      hadoop-project/pom.xml
  92. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
  93. 45 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  94. 79 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java
  95. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/package-info.java
  96. 106 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  97. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
  98. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java
  99. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
  100. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java

+ 17 - 5
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)

+ 10 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java

@@ -20,7 +20,14 @@ package org.apache.hadoop.io.erasurecode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.*;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder;
 
 /**
  * A codec & coder utility to help create raw coders conveniently.
@@ -43,7 +50,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         true, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawEncoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawEncoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureEncoder) rawCoder;
@@ -62,7 +69,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         false, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawDecoderLegacy(numDataUnits, numParityUnits);
+      rawCoder = new RSRawDecoder(numDataUnits, numParityUnits);
     }
 
     return (RawErasureDecoder) rawCoder;

+ 6 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.io.erasurecode.rawcoder.util.CoderUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.GF256;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -35,7 +34,7 @@ import java.util.Arrays;
  * from HDFS-RAID, and also compatible with the native/ISA-L coder.
  */
 @InterfaceAudience.Private
-public class RSRawDecoder2 extends AbstractRawErasureDecoder {
+public class RSRawDecoder extends AbstractRawErasureDecoder {
   //relevant to schema and won't change during decode calls
   private byte[] encodeMatrix;
 
@@ -55,7 +54,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
   private int numErasedDataUnits;
   private boolean[] erasureFlags;
 
-  public RSRawDecoder2(int numDataUnits, int numParityUnits) {
+  public RSRawDecoder(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
     if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
       throw new HadoopIllegalArgumentException(
@@ -64,7 +63,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
 
     int numAllUnits = getNumDataUnits() + numParityUnits;
     encodeMatrix = new byte[numAllUnits * getNumDataUnits()];
-    RSUtil2.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
+    RSUtil.genCauchyMatrix(encodeMatrix, numAllUnits, getNumDataUnits());
     if (isAllowingVerboseDump()) {
       DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, numAllUnits);
     }
@@ -79,7 +78,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
     for (int i = 0; i < getNumDataUnits(); i++) {
       realInputs[i] = inputs[validIndexes[i]];
     }
-    RSUtil2.encodeData(gfTables, realInputs, outputs);
+    RSUtil.encodeData(gfTables, realInputs, outputs);
   }
 
   @Override
@@ -94,7 +93,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
       realInputs[i] = inputs[validIndexes[i]];
       realInputOffsets[i] = inputOffsets[validIndexes[i]];
     }
-    RSUtil2.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
+    RSUtil.encodeData(gfTables, dataLen, realInputs, realInputOffsets,
             outputs, outputOffsets);
   }
 
@@ -131,7 +130,7 @@ public class RSRawDecoder2 extends AbstractRawErasureDecoder {
 
     generateDecodeMatrix(erasedIndexes);
 
-    RSUtil2.initTables(getNumDataUnits(), erasedIndexes.length,
+    RSUtil.initTables(getNumDataUnits(), erasedIndexes.length,
         decodeMatrix, 0, gfTables);
     if (isAllowingVerboseDump()) {
       System.out.println(DumpUtil.bytesToHex(gfTables, -1));

+ 6 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java

@@ -21,7 +21,6 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil;
 import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil2;
 
 import java.nio.ByteBuffer;
 
@@ -32,7 +31,7 @@ import java.nio.ByteBuffer;
  * from HDFS-RAID, and also compatible with the native/ISA-L coder.
  */
 @InterfaceAudience.Private
-public class RSRawEncoder2 extends AbstractRawErasureEncoder {
+public class RSRawEncoder extends AbstractRawErasureEncoder {
   // relevant to schema and won't change during encode calls.
   private byte[] encodeMatrix;
   /**
@@ -41,7 +40,7 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
    */
   private byte[] gfTables;
 
-  public RSRawEncoder2(int numDataUnits, int numParityUnits) {
+  public RSRawEncoder(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
 
     if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
@@ -50,12 +49,12 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
     }
 
     encodeMatrix = new byte[getNumAllUnits() * numDataUnits];
-    RSUtil2.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
+    RSUtil.genCauchyMatrix(encodeMatrix, getNumAllUnits(), numDataUnits);
     if (isAllowingVerboseDump()) {
       DumpUtil.dumpMatrix(encodeMatrix, numDataUnits, getNumAllUnits());
     }
     gfTables = new byte[getNumAllUnits() * numDataUnits * 32];
-    RSUtil2.initTables(numDataUnits, numParityUnits, encodeMatrix,
+    RSUtil.initTables(numDataUnits, numParityUnits, encodeMatrix,
         numDataUnits * numDataUnits, gfTables);
     if (isAllowingVerboseDump()) {
       System.out.println(DumpUtil.bytesToHex(gfTables, -1));
@@ -64,13 +63,13 @@ public class RSRawEncoder2 extends AbstractRawErasureEncoder {
 
   @Override
   protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    RSUtil2.encodeData(gfTables, inputs, outputs);
+    RSUtil.encodeData(gfTables, inputs, outputs);
   }
 
   @Override
   protected void doEncode(byte[][] inputs, int[] inputOffsets,
                           int dataLen, byte[][] outputs, int[] outputOffsets) {
-    RSUtil2.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
+    RSUtil.encodeData(gfTables, dataLen, inputs, inputOffsets, outputs,
         outputOffsets);
   }
 }

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory2.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java

@@ -20,18 +20,18 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
- * A raw coder factory for raw Reed-Solomon coder in Java.
+ * A raw coder factory for the new raw Reed-Solomon coder in Java.
  */
 @InterfaceAudience.Private
-public class RSRawErasureCoderFactory2 implements RawErasureCoderFactory {
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
 
   @Override
   public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
-    return new RSRawEncoder2(numDataUnits, numParityUnits);
+    return new RSRawEncoder(numDataUnits, numParityUnits);
   }
 
   @Override
   public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
-    return new RSRawDecoder2(numDataUnits, numParityUnits);
+    return new RSRawDecoder(numDataUnits, numParityUnits);
   }
 }

+ 147 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java

@@ -19,11 +19,16 @@ package org.apache.hadoop.io.erasurecode.rawcoder.util;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
+import java.nio.ByteBuffer;
+
 /**
- * Utilities for implementing Reed-Solomon code, used by RS coder.
+ * Utilities for implementing Reed-Solomon code, used by RS coder. Some of the
+ * codes are borrowed from ISA-L implementation (C or ASM codes).
  */
 @InterfaceAudience.Private
-public class RSUtil {
+public final class RSUtil {
+
+  private RSUtil(){}
 
   // We always use the byte system (with symbol size 8, field size 256,
   // primitive polynomial 285, and primitive root 2).
@@ -39,4 +44,144 @@ public class RSUtil {
     return primitivePower;
   }
 
+  public static void initTables(int k, int rows, byte[] codingMatrix,
+      int matrixOffset, byte[] gfTables) {
+    int i, j;
+
+    int offset = 0, idx = matrixOffset;
+    for (i = 0; i < rows; i++) {
+      for (j = 0; j < k; j++) {
+        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
+        offset += 32;
+      }
+    }
+  }
+
+  /**
+   * Ported from Intel ISA-L library.
+   */
+  public static void genCauchyMatrix(byte[] a, int m, int k) {
+    // Identity matrix in high position
+    for (int i = 0; i < k; i++) {
+      a[k * i + i] = 1;
+    }
+
+    // For the rest choose 1/(i + j) | i != j
+    int pos = k * k;
+    for (int i = k; i < m; i++) {
+      for (int j = 0; j < k; j++) {
+        a[pos++] = GF256.gfInv((byte) (i ^ j));
+      }
+    }
+  }
+
+  /**
+   * Encode a group of inputs data and generate the outputs. It's also used for
+   * decoding because, in this implementation, encoding and decoding are
+   * unified.
+   *
+   * The algorithm is ported from Intel ISA-L library for compatible. It
+   * leverages Java auto-vectorization support for performance.
+   */
+  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
+      int[] inputOffsets, byte[][] outputs,
+      int[] outputOffsets) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int l, i, j, iPos, oPos;
+    byte[] input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = inputOffsets[j];
+        oPos = outputOffsets[l];
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        /**
+         * Purely for performance, assuming we can use 8 bytes in the SIMD
+         * instruction. Subject to be improved.
+         */
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
+          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
+          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
+          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
+          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
+          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
+          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
+          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
+        }
+
+        /**
+         * For the left bytes, do it one by one.
+         */
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output[oPos] ^= tableLine[0xff & input[iPos]];
+        }
+      }
+    }
+  }
+
+  /**
+   * See above. Try to use the byte[] version when possible.
+   */
+  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
+      ByteBuffer[] outputs) {
+    int numInputs = inputs.length;
+    int numOutputs = outputs.length;
+    int dataLen = inputs[0].remaining();
+    int l, i, j, iPos, oPos;
+    ByteBuffer input, output;
+    byte s;
+    final int times = dataLen / 8;
+    final int extra = dataLen - dataLen % 8;
+    byte[] tableLine;
+
+    for (l = 0; l < numOutputs; l++) {
+      output = outputs[l];
+
+      for (j = 0; j < numInputs; j++) {
+        input = inputs[j];
+        iPos = input.position();
+        oPos = output.position();
+
+        s = gfTables[j * 32 + l * numInputs * 32 + 1];
+        tableLine = GF256.gfMulTab()[s & 0xff];
+
+        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
+          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
+              tableLine[0xff & input.get(iPos + 0)]));
+          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
+              tableLine[0xff & input.get(iPos + 1)]));
+          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
+              tableLine[0xff & input.get(iPos + 2)]));
+          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
+              tableLine[0xff & input.get(iPos + 3)]));
+          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
+              tableLine[0xff & input.get(iPos + 4)]));
+          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
+              tableLine[0xff & input.get(iPos + 5)]));
+          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
+              tableLine[0xff & input.get(iPos + 6)]));
+          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
+              tableLine[0xff & input.get(iPos + 7)]));
+        }
+
+        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
+          output.put(oPos, (byte) (output.get(oPos) ^
+              tableLine[0xff & input.get(iPos)]));
+        }
+      }
+    }
+  }
+
 }

+ 0 - 172
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil2.java

@@ -1,172 +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.
- */
-package org.apache.hadoop.io.erasurecode.rawcoder.util;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.nio.ByteBuffer;
-
-/**
- * Utilities for implementing Reed-Solomon code, used by RS2 coder. Some of the
- * codes are borrowed from ISA-L implementation (C or ASM codes).
- */
-@InterfaceAudience.Private
-public final class RSUtil2 {
-
-  private RSUtil2() { }
-
-  public static void initTables(int k, int rows, byte[] codingMatrix,
-                                int matrixOffset, byte[] gfTables) {
-    int i, j;
-
-    int offset = 0, idx = matrixOffset;
-    for (i = 0; i < rows; i++) {
-      for (j = 0; j < k; j++) {
-        GF256.gfVectMulInit(codingMatrix[idx++], gfTables, offset);
-        offset += 32;
-      }
-    }
-  }
-
-  /**
-   * Ported from Intel ISA-L library.
-   */
-  public static void genCauchyMatrix(byte[] a, int m, int k) {
-    // Identity matrix in high position
-    for (int i = 0; i < k; i++) {
-      a[k * i + i] = 1;
-    }
-
-    // For the rest choose 1/(i + j) | i != j
-    int pos = k * k;
-    for (int i = k; i < m; i++) {
-      for (int j = 0; j < k; j++) {
-        a[pos++] = GF256.gfInv((byte) (i ^ j));
-      }
-    }
-  }
-
-  /**
-   * Encode a group of inputs data and generate the outputs. It's also used for
-   * decoding because, in this implementation, encoding and decoding are
-   * unified.
-   *
-   * The algorithm is ported from Intel ISA-L library for compatible. It
-   * leverages Java auto-vectorization support for performance.
-   */
-  public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs,
-                                int[] inputOffsets, byte[][] outputs,
-                                int[] outputOffsets) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int l, i, j, iPos, oPos;
-    byte[] input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = inputOffsets[j];
-        oPos = outputOffsets[l];
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        /**
-         * Purely for performance, assuming we can use 8 bytes in the SIMD
-         * instruction. Subject to be improved.
-         */
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output[oPos + 0] ^= tableLine[0xff & input[iPos + 0]];
-          output[oPos + 1] ^= tableLine[0xff & input[iPos + 1]];
-          output[oPos + 2] ^= tableLine[0xff & input[iPos + 2]];
-          output[oPos + 3] ^= tableLine[0xff & input[iPos + 3]];
-          output[oPos + 4] ^= tableLine[0xff & input[iPos + 4]];
-          output[oPos + 5] ^= tableLine[0xff & input[iPos + 5]];
-          output[oPos + 6] ^= tableLine[0xff & input[iPos + 6]];
-          output[oPos + 7] ^= tableLine[0xff & input[iPos + 7]];
-        }
-
-        /**
-         * For the left bytes, do it one by one.
-         */
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output[oPos] ^= tableLine[0xff & input[iPos]];
-        }
-      }
-    }
-  }
-
-  /**
-   * See above. Try to use the byte[] version when possible.
-   */
-  public static void encodeData(byte[] gfTables, ByteBuffer[] inputs,
-                                ByteBuffer[] outputs) {
-    int numInputs = inputs.length;
-    int numOutputs = outputs.length;
-    int dataLen = inputs[0].remaining();
-    int l, i, j, iPos, oPos;
-    ByteBuffer input, output;
-    byte s;
-    final int times = dataLen / 8;
-    final int extra = dataLen - dataLen % 8;
-    byte[] tableLine;
-
-    for (l = 0; l < numOutputs; l++) {
-      output = outputs[l];
-
-      for (j = 0; j < numInputs; j++) {
-        input = inputs[j];
-        iPos = input.position();
-        oPos = output.position();
-
-        s = gfTables[j * 32 + l * numInputs * 32 + 1];
-        tableLine = GF256.gfMulTab()[s & 0xff];
-
-        for (i = 0; i < times; i++, iPos += 8, oPos += 8) {
-          output.put(oPos + 0, (byte) (output.get(oPos + 0) ^
-              tableLine[0xff & input.get(iPos + 0)]));
-          output.put(oPos + 1, (byte) (output.get(oPos + 1) ^
-              tableLine[0xff & input.get(iPos + 1)]));
-          output.put(oPos + 2, (byte) (output.get(oPos + 2) ^
-              tableLine[0xff & input.get(iPos + 2)]));
-          output.put(oPos + 3, (byte) (output.get(oPos + 3) ^
-              tableLine[0xff & input.get(iPos + 3)]));
-          output.put(oPos + 4, (byte) (output.get(oPos + 4) ^
-              tableLine[0xff & input.get(iPos + 4)]));
-          output.put(oPos + 5, (byte) (output.get(oPos + 5) ^
-              tableLine[0xff & input.get(iPos + 5)]));
-          output.put(oPos + 6, (byte) (output.get(oPos + 6) ^
-              tableLine[0xff & input.get(iPos + 6)]));
-          output.put(oPos + 7, (byte) (output.get(oPos + 7) ^
-              tableLine[0xff & input.get(iPos + 7)]));
-        }
-
-        for (i = extra; i < dataLen; i++, iPos++, oPos++) {
-          output.put(oPos, (byte) (output.get(oPos) ^
-              tableLine[0xff & input.get(iPos)]));
-        }
-      }
-    }
-  }
-}

+ 3 - 106
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -1120,12 +1120,11 @@ public class Client {
         if (LOG.isDebugEnabled())
           LOG.debug(getName() + " got value #" + callId);
 
-        Call call = calls.get(callId);
         RpcStatusProto status = header.getStatus();
         if (status == RpcStatusProto.SUCCESS) {
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           value.readFields(in);                 // read value
-          calls.remove(callId);
+          final Call call = calls.remove(callId);
           call.setRpcResponse(value);
           
           // verify that length was correct
@@ -1157,7 +1156,7 @@ public class Client {
           }
           RemoteException re = new RemoteException(exceptionClassName, errorMsg, erCode);
           if (status == RpcStatusProto.ERROR) {
-            calls.remove(callId);
+            final Call call = calls.remove(callId);
             call.setException(re);
           } else if (status == RpcStatusProto.FATAL) {
             // Close the connection
@@ -1288,85 +1287,6 @@ public class Client {
     clientExcecutorFactory.unrefAndCleanup();
   }
 
-  /**
-   * Same as {@link #call(RPC.RpcKind, Writable, ConnectionId)}
-   *  for RPC_BUILTIN
-   */
-  public Writable call(Writable param, InetSocketAddress address)
-      throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(address, null, null, 0,
-        conf);
-    return call(RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-
-  /**
-   * Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress,
-   * Class, UserGroupInformation, int, Configuration)}
-   * except that rpcKind is writable.
-   */
-  public Writable call(Writable param, InetSocketAddress addr,
-      Class<?> protocol, UserGroupInformation ticket,
-      int rpcTimeout, Configuration conf) throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-  
-  /**
-   * Same as {@link #call(Writable, InetSocketAddress,
-   * Class, UserGroupInformation, int, Configuration)}
-   * except that specifying serviceClass.
-   */
-  public Writable call(Writable param, InetSocketAddress addr,
-      Class<?> protocol, UserGroupInformation ticket,
-      int rpcTimeout, int serviceClass, Configuration conf)
-      throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass);
-  }
-
-  /**
-   * Make a call, passing <code>param</code>, to the IPC server running at
-   * <code>address</code> which is servicing the <code>protocol</code> protocol,
-   * with the <code>ticket</code> credentials, <code>rpcTimeout</code> as
-   * timeout and <code>conf</code> as conf for this connection, returning the
-   * value. Throws exceptions if there are network problems or if the remote
-   * code threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable param, InetSocketAddress addr, 
-                       Class<?> protocol, UserGroupInformation ticket,
-                       int rpcTimeout, Configuration conf) throws IOException {
-    ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
-        ticket, rpcTimeout, conf);
-    return call(rpcKind, param, remoteId);
-  }
-  
-  /**
-   * Same as {link {@link #call(RPC.RpcKind, Writable, ConnectionId)}
-   * except the rpcKind is RPC_BUILTIN
-   */
-  public Writable call(Writable param, ConnectionId remoteId)
-      throws IOException {
-     return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
-  }
-  
-  /**
-   * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
-   * <code>remoteId</code>, returning the rpc respond.
-   *
-   * @param rpcKind
-   * @param rpcRequest -  contains serialized method and method parameters
-   * @param remoteId - the target rpc server
-   * @returns the rpc response
-   * Throws exceptions if there are network problems or if the remote code 
-   * threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
-      ConnectionId remoteId) throws IOException {
-    return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT);
-  }
-
   /** 
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc respond.
@@ -1387,23 +1307,6 @@ public class Client {
       fallbackToSimpleAuth);
   }
 
-  /**
-   * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
-   * <code>remoteId</code>, returning the rpc response.
-   * 
-   * @param rpcKind
-   * @param rpcRequest -  contains serialized method and method parameters
-   * @param remoteId - the target rpc server
-   * @param serviceClass - service class for RPC
-   * @returns the rpc response
-   * Throws exceptions if there are network problems or if the remote code 
-   * threw an exception.
-   */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
-      ConnectionId remoteId, int serviceClass) throws IOException {
-    return call(rpcKind, rpcRequest, remoteId, serviceClass, null);
-  }
-
   /**
    * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
    * <code>remoteId</code>, returning the rpc response.
@@ -1418,7 +1321,7 @@ public class Client {
    * Throws exceptions if there are network problems or if the remote code
    * threw an exception.
    */
-  public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
+  Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
       ConnectionId remoteId, int serviceClass,
       AtomicBoolean fallbackToSimpleAuth) throws IOException {
     final Call call = createCall(rpcKind, rpcRequest);
@@ -1620,12 +1523,6 @@ public class Client {
       return saslQop;
     }
     
-    static ConnectionId getConnectionId(InetSocketAddress addr,
-        Class<?> protocol, UserGroupInformation ticket, int rpcTimeout,
-        Configuration conf) throws IOException {
-      return getConnectionId(addr, protocol, ticket, rpcTimeout, null, conf);
-    }
-
     /**
      * Returns a ConnectionId object. 
      * @param addr Remote address for the connection.

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -221,7 +221,7 @@ public class WritableRpcEngine implements RpcEngine {
                    int rpcTimeout, AtomicBoolean fallbackToSimpleAuth)
         throws IOException {
       this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
-          ticket, rpcTimeout, conf);
+          ticket, rpcTimeout, null, conf);
       this.client = CLIENTS.getClient(conf, factory);
       this.fallbackToSimpleAuth = fallbackToSimpleAuth;
     }

+ 16 - 1
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()) {

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java

@@ -399,7 +399,7 @@ public abstract class AbstractService implements Service {
   }
 
   @Override
-  public synchronized Configuration getConfig() {
+  public Configuration getConfig() {
     return config;
   }
 

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -717,6 +717,10 @@ public abstract class Shell {
     } catch (IOException ioe) {
       LOG.warn("Bash is not supported by the OS", ioe);
       supported = false;
+    } catch (SecurityException se) {
+      LOG.info("Bash execution is not allowed by the JVM " +
+          "security manager.Considering it not supported.");
+      supported = false;
     }
 
     return supported;
@@ -744,7 +748,11 @@ public abstract class Shell {
     } catch (IOException ioe) {
       LOG.debug("setsid is not available on this machine. So not using it.");
       setsidSupported = false;
-    }  catch (Error err) {
+    } catch (SecurityException se) {
+      LOG.debug("setsid is not allowed to run by the JVM "+
+          "security manager. So not using it.");
+      setsidSupported = false;
+    } catch (Error err) {
       if (err.getMessage() != null
           && err.getMessage().contains("posix_spawn is not " +
           "a supported process launch mechanism")

+ 9 - 2
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" />

+ 100 - 0
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);
+}
+

+ 40 - 0
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_

+ 6 - 233
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);
 }

+ 0 - 18
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/erasure_code.h → hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/erasure_code.h

@@ -37,24 +37,6 @@
  *
  */
 
-/**
- * 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.
  *

+ 229 - 0
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;
+}

+ 88 - 0
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_

+ 54 - 0
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);
+}

+ 0 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/include/gf_util.h → hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/gf_util.h


+ 148 - 0
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;
+}

+ 149 - 0
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_

+ 98 - 0
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];
+  }
+}

+ 42 - 0
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_

+ 2 - 6
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/erasure_code_native.c → hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/jni_erasure_code_native.c

@@ -21,7 +21,7 @@
 #include <string.h>
 
 #include "org_apache_hadoop.h"
-#include "../include/erasure_code.h"
+#include "jni_common.h"
 #include "org_apache_hadoop_io_erasurecode_ErasureCodeNative.h"
 
 #ifdef UNIX
@@ -31,11 +31,7 @@
 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);
-  }
+  loadLib(env);
 }
 
 JNIEXPORT jstring JNICALL

+ 72 - 0
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);
+}

+ 66 - 0
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);
+}

+ 0 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/coder/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h → hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/erasurecode/org_apache_hadoop_io_erasurecode_ErasureCodeNative.h


+ 37 - 0
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

+ 37 - 0
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

+ 61 - 246
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;
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -51,7 +51,7 @@ public class TestHHXORErasureCoder extends TestHHErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);

+ 4 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -58,20 +58,20 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_p1() {
     prepare(null, 10, 4, new int[]{}, new int[]{1});
     testCoding(true);
     testCoding(true);
   }
-  
+
   @Test
   public void testCodingDirectBuffer_10x4_erasing_d2() {
     prepare(null, 10, 4, new int[] {2}, new int[] {});

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder2.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java

@@ -22,12 +22,12 @@ import org.junit.Before;
 /**
  * Test the new raw Reed-solomon coder implemented in Java.
  */
-public class TestRSRawCoder2 extends TestRSRawCoderBase {
+public class TestRSRawCoder extends TestRSRawCoderBase {
 
   @Before
   public void setup() {
-    this.encoderClass = RSRawEncoder2.class;
-    this.decoderClass = RSRawDecoder2.class;
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
     setAllowDump(false);
   }
 }

+ 58 - 43
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java

@@ -55,7 +55,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.net.SocketFactory;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -63,7 +62,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
@@ -78,9 +76,9 @@ import org.apache.hadoop.ipc.Server.Connection;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assume;
@@ -91,6 +89,7 @@ import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import com.google.common.base.Supplier;
 import com.google.common.primitives.Bytes;
 import com.google.common.primitives.Ints;
 
@@ -122,6 +121,33 @@ public class TestIPC {
   /** Directory where we can count open file descriptors on Linux */
   private static final File FD_DIR = new File("/proc/self/fd");
 
+  static ConnectionId getConnectionId(InetSocketAddress addr, int rpcTimeout,
+      Configuration conf) throws IOException {
+    return ConnectionId.getConnectionId(addr, null, null, rpcTimeout, null,
+        conf);
+  }
+
+  static Writable call(Client client, InetSocketAddress addr,
+      int serviceClass, Configuration conf) throws IOException {
+    final LongWritable param = new LongWritable(RANDOM.nextLong());
+    final ConnectionId remoteId = getConnectionId(addr, MIN_SLEEP_TIME, conf);
+    return client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass,
+        null);
+  }
+
+  static LongWritable call(Client client, long param, InetSocketAddress addr,
+      Configuration conf) throws IOException {
+    return call(client, new LongWritable(param), addr, 0, conf);
+  }
+
+  static LongWritable call(Client client, LongWritable param,
+      InetSocketAddress addr, int rpcTimeout, Configuration conf)
+          throws IOException {
+    final ConnectionId remoteId = getConnectionId(addr, rpcTimeout, conf);
+    return (LongWritable)client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId,
+        RPC.RPC_SERVICE_CLASS_DEFAULT, null);
+  }
+
   private static class TestServer extends Server {
     // Tests can set callListener to run a piece of code each time the server
     // receives a call.  This code executes on the server thread, so it has
@@ -183,10 +209,9 @@ public class TestIPC {
     public void run() {
       for (int i = 0; i < count; i++) {
         try {
-          LongWritable param = new LongWritable(RANDOM.nextLong());
-          LongWritable value =
-            (LongWritable)client.call(param, server, null, null, 0, conf);
-          if (!param.equals(value)) {
+          final long param = RANDOM.nextLong();
+          LongWritable value = call(client, param, server, conf);
+          if (value.get() != param) {
             LOG.fatal("Call failed!");
             failed = true;
             break;
@@ -226,9 +251,8 @@ public class TestIPC {
     @Override
     public Object invoke(Object proxy, Method method, Object[] args)
         throws Throwable {
-      LongWritable param = new LongWritable(RANDOM.nextLong());
-      LongWritable value = (LongWritable) client.call(param,
-          NetUtils.getConnectAddress(server), null, null, 0, conf);
+      LongWritable value = call(client, RANDOM.nextLong(),
+          NetUtils.getConnectAddress(server), conf);
       return returnValue(value);
     }
 
@@ -298,8 +322,7 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Expected an exception to have been thrown");
     } catch (IOException e) {
       String message = e.getMessage();
@@ -412,7 +435,7 @@ public class TestIPC {
       LongWritable param = clientParamClass.newInstance();
 
       try {
-        client.call(param, addr, null, null, 0, conf);
+        call(client, param, addr, 0, conf);
         fail("Expected an exception to have been thrown");
       } catch (Throwable t) {
         assertExceptionContains(t, "Injected fault");
@@ -422,7 +445,7 @@ public class TestIPC {
       // ie the internal state of the client or server should not be broken
       // by the failed call
       WRITABLE_FAULTS_ENABLED = false;
-      client.call(param, addr, null, null, 0, conf);
+      call(client, param, addr, 0, conf);
       
     } finally {
       client.stop();
@@ -536,8 +559,7 @@ public class TestIPC {
     
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Expected an exception to have been thrown");
     } catch (IOException e) {
       assertTrue(e.getMessage().contains("Injected fault"));
@@ -574,8 +596,7 @@ public class TestIPC {
       // Call should fail due to injected exception.
       InetSocketAddress address = NetUtils.getConnectAddress(server);
       try {
-        client.call(new LongWritable(RANDOM.nextLong()),
-                address, null, null, 0, conf);
+        call(client, RANDOM.nextLong(), address, conf);
         fail("Expected an exception to have been thrown");
       } catch (Exception e) {
         LOG.info("caught expected exception", e);
@@ -586,8 +607,7 @@ public class TestIPC {
       // (i.e. it should not have cached a half-constructed connection)
   
       Mockito.reset(spyFactory);
-      client.call(new LongWritable(RANDOM.nextLong()),
-          address, null, null, 0, conf);
+      call(client, RANDOM.nextLong(), address, conf);
     } finally {
       client.stop();
       server.stop();
@@ -605,15 +625,15 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     // set timeout to be less than MIN_SLEEP_TIME
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              addr, null, null, MIN_SLEEP_TIME/2, conf);
+      call(client, new LongWritable(RANDOM.nextLong()), addr,
+          MIN_SLEEP_TIME / 2, conf);
       fail("Expected an exception to have been thrown");
     } catch (SocketTimeoutException e) {
       LOG.info("Get a SocketTimeoutException ", e);
     }
     // set timeout to be bigger than 3*ping interval
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
+    call(client, new LongWritable(RANDOM.nextLong()), addr,
+        3 * PING_INTERVAL + MIN_SLEEP_TIME, conf);
     client.stop();
   }
 
@@ -629,8 +649,8 @@ public class TestIPC {
     Client client = new Client(LongWritable.class, conf);
     // set the rpc timeout to twice the MIN_SLEEP_TIME
     try {
-      client.call(new LongWritable(RANDOM.nextLong()),
-              addr, null, null, MIN_SLEEP_TIME*2, conf);
+      call(client, new LongWritable(RANDOM.nextLong()), addr,
+          MIN_SLEEP_TIME * 2, conf);
       fail("Expected an exception to have been thrown");
     } catch (SocketTimeoutException e) {
       LOG.info("Get a SocketTimeoutException ", e);
@@ -743,8 +763,8 @@ public class TestIPC {
         public void run() {
           Client client = new Client(LongWritable.class, conf);
           try {
-            client.call(new LongWritable(Thread.currentThread().getId()),
-                addr, null, null, 60000, conf);
+            call(client, new LongWritable(Thread.currentThread().getId()),
+                addr, 60000, conf);
           } catch (Throwable e) {
             LOG.error(e);
             failures.incrementAndGet();
@@ -875,8 +895,7 @@ public class TestIPC {
           public void run() {
             Client client = new Client(LongWritable.class, clientConf);
             try {
-              client.call(new LongWritable(Thread.currentThread().getId()),
-                  addr, null, null, 0, clientConf);
+              call(client, Thread.currentThread().getId(), addr, clientConf);
               callReturned.countDown();
               Thread.sleep(10000);
             } catch (IOException e) {
@@ -931,16 +950,15 @@ public class TestIPC {
       }
     }
   }
-  
+
   /**
    * Make a call from a client and verify if header info is changed in server side
    */
-  private void callAndVerify(Server server, InetSocketAddress addr,
+  private static void callAndVerify(Server server, InetSocketAddress addr,
       int serviceClass, boolean noChanged) throws IOException{
     Client client = new Client(LongWritable.class, conf);
 
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
+    call(client, addr, serviceClass, conf);
     Connection connection = server.getConnections()[0];
     int serviceClass2 = connection.getServiceClass();
     assertFalse(noChanged ^ serviceClass == serviceClass2);
@@ -956,13 +974,11 @@ public class TestIPC {
 
     // start client
     Client client = new Client(LongWritable.class, conf);
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, 0, conf);
+    call(client, addr, 0, conf);
     client.stop();
  
     // This call should throw IOException.
-    client.call(new LongWritable(RANDOM.nextLong()),
-        addr, null, null, MIN_SLEEP_TIME, 0, conf);
+    call(client, addr, 0, conf);
   }
 
   /**
@@ -992,7 +1008,7 @@ public class TestIPC {
   @Test(timeout=30000)
   public void testInterrupted() {
     Client client = new Client(LongWritable.class, conf);
-    client.getClientExecutor().submit(new Runnable() {
+    Client.getClientExecutor().submit(new Runnable() {
       public void run() {
         while(true);
       }
@@ -1007,7 +1023,7 @@ public class TestIPC {
       Assert.fail("The Client did not interrupt after handling an Interrupted Exception");
     }
     // Clear Thread interrupt
-    Thread.currentThread().interrupted();
+    Thread.interrupted();
   }
 
   private long countOpenFileDescriptors() {
@@ -1363,11 +1379,10 @@ public class TestIPC {
       int maxTimeoutRetries) throws IOException {
     SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
     doThrow(new ConnectTimeoutException("fake")).when(mockFactory).createSocket();
-    Client client = new Client(IntWritable.class, conf, mockFactory);
+    Client client = new Client(LongWritable.class, conf, mockFactory);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
     try {
-      client.call(new IntWritable(RANDOM.nextInt()), address, null, null, 0,
-          conf);
+      call(client, RANDOM.nextLong(), address, conf);
       fail("Not throwing the SocketTimeoutException");
     } catch (SocketTimeoutException e) {
       Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))

+ 19 - 11
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.Client.ConnectionId;
+import org.apache.hadoop.ipc.RPC.RpcKind;
 import org.apache.hadoop.ipc.Server.Call;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.Assert;
@@ -66,6 +68,14 @@ public class TestIPCServerResponder {
       BYTES[i] = (byte) ('a' + (i % 26));
   }
 
+  static Writable call(Client client, Writable param,
+      InetSocketAddress address) throws IOException {
+    final ConnectionId remoteId = ConnectionId.getConnectionId(address, null,
+        null, 0, null, conf);
+    return client.call(RpcKind.RPC_BUILTIN, param, remoteId,
+        RPC.RPC_SERVICE_CLASS_DEFAULT, null);
+  }
+
   private static class TestServer extends Server {
 
     private boolean sleep;
@@ -113,7 +123,7 @@ public class TestIPCServerResponder {
           byte[] bytes = new byte[byteSize];
           System.arraycopy(BYTES, 0, bytes, 0, byteSize);
           Writable param = new BytesWritable(bytes);
-          client.call(param, address);
+          call(client, param, address);
           Thread.sleep(RANDOM.nextInt(20));
         } catch (Exception e) {
           LOG.fatal("Caught Exception", e);
@@ -209,17 +219,16 @@ public class TestIPCServerResponder {
 
     // calls should return immediately, check the sequence number is
     // increasing
-    assertEquals(0,
-        ((IntWritable)client.call(wait0, address)).get());
-    assertEquals(1,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(0, ((IntWritable)call(client, wait0, address)).get());
+    assertEquals(1, ((IntWritable)call(client, wait0, address)).get());
 
     // do a call in the background that will have a deferred response
     final ExecutorService exec = Executors.newCachedThreadPool();
     Future<Integer> future1 = exec.submit(new Callable<Integer>() {
       @Override
       public Integer call() throws IOException {
-        return ((IntWritable)client.call(wait1, address)).get();
+        return ((IntWritable)TestIPCServerResponder.call(
+            client, wait1, address)).get();
       }
     });
     // make sure it blocked
@@ -237,14 +246,14 @@ public class TestIPCServerResponder {
 
     // proves the handler isn't tied up, and that the prior sequence number
     // was consumed
-    assertEquals(3,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(3, ((IntWritable)call(client, wait0, address)).get());
 
     // another call with wait count of 2
     Future<Integer> future2 = exec.submit(new Callable<Integer>() {
       @Override
       public Integer call() throws IOException {
-        return ((IntWritable)client.call(wait2, address)).get();
+        return ((IntWritable)TestIPCServerResponder.call(
+            client, wait2, address)).get();
       }
     });
     // make sure it blocked
@@ -286,8 +295,7 @@ public class TestIPCServerResponder {
     assertFalse(future2.isDone());
 
     // call should return immediately
-    assertEquals(5,
-        ((IntWritable)client.call(wait0, address)).get());
+    assertEquals(5, ((IntWritable)call(client, wait0, address)).get());
 
     // trigger last waiting call
     waitingCalls[1].sendResponse();

+ 9 - 9
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java

@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.SIMPLE;
 import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.TOKEN;
@@ -42,16 +42,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -454,7 +451,12 @@ public class TestSaslRPC {
       }
     }
   }
-  
+
+  static ConnectionId getConnectionId(Configuration conf) throws IOException {
+    return ConnectionId.getConnectionId(new InetSocketAddress(0),
+        TestSaslProtocol.class, null, 0, null, conf);
+  }
+
   @Test
   public void testPingInterval() throws Exception {
     Configuration newConf = new Configuration(conf);
@@ -464,14 +466,12 @@ public class TestSaslRPC {
 
     // set doPing to true
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    ConnectionId remoteId = ConnectionId.getConnectionId(
-        new InetSocketAddress(0), TestSaslProtocol.class, null, 0, newConf);
+    ConnectionId remoteId = getConnectionId(newConf);
     assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT,
         remoteId.getPingInterval());
     // set doPing to false
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, false);
-    remoteId = ConnectionId.getConnectionId(
-        new InetSocketAddress(0), TestSaslProtocol.class, null, 0, newConf);
+    remoteId = getConnectionId(newConf);
     assertEquals(0, remoteId.getPingInterval());
   }
   

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java

@@ -162,7 +162,9 @@ public class Block implements Writable, Comparable<Block> {
    */
   public static String toString(final Block b) {
     StringBuilder sb = new StringBuilder();
-    b.appendStringTo(sb);
+    sb.append(BLOCK_FILE_PREFIX).
+       append(b.blockId).append("_").
+       append(b.generationStamp);
     return sb.toString();
   }
 

+ 138 - 130
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -149,7 +149,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   private volatile long pendingReplicationBlocksCount = 0L;
   private volatile long corruptReplicaBlocksCount = 0L;
-  private volatile long underReplicatedBlocksCount = 0L;
+  private volatile long lowRedundancyBlocksCount = 0L;
   private volatile long scheduledReplicationBlocksCount = 0L;
 
   /** flag indicating whether replication queues have been initialized */
@@ -166,7 +166,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
   /** Used by metrics */
   public long getUnderReplicatedBlocksCount() {
-    return underReplicatedBlocksCount;
+    return lowRedundancyBlocksCount;
   }
   /** Used by metrics */
   public long getCorruptReplicaBlocksCount() {
@@ -250,9 +250,10 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Store set of Blocks that need to be replicated 1 or more times.
-   * We also store pending replication-orders.
+   * We also store pending reconstruction-orders.
    */
-  public final UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
+  public final LowRedundancyBlocks neededReconstruction =
+      new LowRedundancyBlocks();
 
   @VisibleForTesting
   final PendingReplicationBlocks pendingReplications;
@@ -294,20 +295,20 @@ public class BlockManager implements BlockStatsMXBean {
   private boolean shouldPostponeBlocksFromFuture = false;
 
   /**
-   * Process replication queues asynchronously to allow namenode safemode exit
-   * and failover to be faster. HDFS-5496
+   * Process reconstruction queues asynchronously to allow namenode safemode
+   * exit and failover to be faster. HDFS-5496.
    */
-  private Daemon replicationQueuesInitializer = null;
+  private Daemon reconstructionQueuesInitializer = null;
   /**
-   * Number of blocks to process asychronously for replication queues
+   * Number of blocks to process asychronously for reconstruction queues
    * initialization once aquired the namesystem lock. Remaining blocks will be
    * processed again after aquiring lock again.
    */
   private int numBlocksPerIteration;
   /**
-   * Progress of the Replication queues initialisation.
+   * Progress of the Reconstruction queues initialisation.
    */
-  private double replicationQueuesInitProgress = 0.0;
+  private double reconstructionQueuesInitProgress = 0.0;
 
   /** for block replicas placement */
   private BlockPlacementPolicies placementPolicies;
@@ -576,12 +577,12 @@ public class BlockManager implements BlockStatsMXBean {
     out.println("Live Datanodes: " + live.size());
     out.println("Dead Datanodes: " + dead.size());
     //
-    // Dump contents of neededReplication
+    // Dump contents of neededReconstruction
     //
-    synchronized (neededReplications) {
-      out.println("Metasave: Blocks waiting for replication: " + 
-                  neededReplications.size());
-      for (Block block : neededReplications) {
+    synchronized (neededReconstruction) {
+      out.println("Metasave: Blocks waiting for reconstruction: "
+          + neededReconstruction.size());
+      for (Block block : neededReconstruction) {
         dumpBlockMeta(block, out);
       }
     }
@@ -616,7 +617,7 @@ public class BlockManager implements BlockStatsMXBean {
     // source node returned is not used
     chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        new LinkedList<Byte>(), UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Byte>(), LowRedundancyBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -849,9 +850,9 @@ public class BlockManager implements BlockStatsMXBean {
     // is happening
     bc.convertLastBlockToUC(lastBlock, targets);
 
-    // Remove block from replication queue.
+    // Remove block from reconstruction queue.
     NumberReplicas replicas = countNodes(lastBlock);
-    neededReplications.remove(lastBlock, replicas.liveReplicas(),
+    neededReconstruction.remove(lastBlock, replicas.liveReplicas(),
         replicas.readOnlyReplicas(),
         replicas.decommissionedAndDecommissioning(), getReplication(lastBlock));
     pendingReplications.remove(lastBlock);
@@ -1365,8 +1366,8 @@ public class BlockManager implements BlockStatsMXBean {
       // the block is over-replicated so invalidate the replicas immediately
       invalidateBlock(b, node, numberOfReplicas);
     } else if (isPopulatingReplQueues()) {
-      // add the block to neededReplication
-      updateNeededReplications(b.getStored(), -1, 0);
+      // add the block to neededReconstruction
+      updateNeededReconstructions(b.getStored(), -1, 0);
     }
   }
 
@@ -1418,13 +1419,13 @@ public class BlockManager implements BlockStatsMXBean {
   
   void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
-    underReplicatedBlocksCount = neededReplications.size();
+    lowRedundancyBlocksCount = neededReconstruction.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
   }
 
-  /** Return number of under-replicated but not missing blocks */
+  /** Return number of low redundancy blocks but not missing blocks. */
   public int getUnderReplicatedNotMissingBlocks() {
-    return neededReplications.getUnderReplicatedBlockCount();
+    return neededReconstruction.getLowRedundancyBlockCount();
   }
   
   /**
@@ -1452,25 +1453,26 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Scan blocks in {@link #neededReplications} and assign reconstruction
+   * Scan blocks in {@link #neededReconstruction} and assign reconstruction
    * (replication or erasure coding) work to data-nodes they belong to.
    *
    * The number of process blocks equals either twice the number of live
-   * data-nodes or the number of under-replicated blocks whichever is less.
+   * data-nodes or the number of low redundancy blocks whichever is less.
    *
-   * @return number of blocks scheduled for replication during this iteration.
+   * @return number of blocks scheduled for reconstruction during this
+   *         iteration.
    */
   int computeBlockReconstructionWork(int blocksToProcess) {
-    List<List<BlockInfo>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReconstruct = null;
     namesystem.writeLock();
     try {
-      // Choose the blocks to be replicated
-      blocksToReplicate = neededReplications
-          .chooseUnderReplicatedBlocks(blocksToProcess);
+      // Choose the blocks to be reconstructed
+      blocksToReconstruct = neededReconstruction
+          .chooseLowRedundancyBlocks(blocksToProcess);
     } finally {
       namesystem.writeUnlock();
     }
-    return computeReconstructionWorkForBlocks(blocksToReplicate);
+    return computeReconstructionWorkForBlocks(blocksToReconstruct);
   }
 
   /**
@@ -1489,7 +1491,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Step 1: categorize at-risk blocks into replication and EC tasks
     namesystem.writeLock();
     try {
-      synchronized (neededReplications) {
+      synchronized (neededReconstruction) {
         for (int priority = 0; priority < blocksToReconstruct
             .size(); priority++) {
           for (BlockInfo block : blocksToReconstruct.get(priority)) {
@@ -1533,7 +1535,7 @@ public class BlockManager implements BlockStatsMXBean {
           continue;
         }
 
-        synchronized (neededReplications) {
+        synchronized (neededReconstruction) {
           if (validateReconstructionWork(rw)) {
             scheduledWork++;
           }
@@ -1544,7 +1546,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     if (blockLog.isDebugEnabled()) {
-      // log which blocks have been scheduled for replication
+      // log which blocks have been scheduled for reconstruction
       for(BlockReconstructionWork rw : reconWork){
         DatanodeStorageInfo[] targets = rw.getTargets();
         if (targets != null && targets.length != 0) {
@@ -1558,8 +1560,9 @@ public class BlockManager implements BlockStatsMXBean {
         }
       }
 
-      blockLog.debug("BLOCK* neededReplications = {} pendingReplications = {}",
-          neededReplications.size(), pendingReplications.size());
+      blockLog.debug(
+          "BLOCK* neededReconstruction = {} pendingReplications = {}",
+          neededReconstruction.size(), pendingReplications.size());
     }
 
     return scheduledWork;
@@ -1576,8 +1579,8 @@ public class BlockManager implements BlockStatsMXBean {
       int priority) {
     // skip abandoned block or block reopened for append
     if (block.isDeleted() || !block.isCompleteOrCommitted()) {
-      // remove from neededReplications
-      neededReplications.remove(block, priority);
+      // remove from neededReconstruction
+      neededReconstruction.remove(block, priority);
       return null;
     }
 
@@ -1605,8 +1608,8 @@ public class BlockManager implements BlockStatsMXBean {
     int pendingNum = pendingReplications.getNumReplicas(block);
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
         requiredReplication)) {
-      neededReplications.remove(block, priority);
-      blockLog.debug("BLOCK* Removing {} from neededReplications as" +
+      neededReconstruction.remove(block, priority);
+      blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
           " it has enough replicas", block);
       return null;
     }
@@ -1662,7 +1665,7 @@ public class BlockManager implements BlockStatsMXBean {
     // Recheck since global lock was released
     // skip abandoned block or block reopened for append
     if (block.isDeleted() || !block.isCompleteOrCommitted()) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
       rw.resetTargets();
       return false;
     }
@@ -1673,7 +1676,7 @@ public class BlockManager implements BlockStatsMXBean {
     final int pendingNum = pendingReplications.getNumReplicas(block);
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum,
         requiredReplication)) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
       rw.resetTargets();
       blockLog.debug("BLOCK* Removing {} from neededReplications as" +
           " it has enough replicas", block);
@@ -1705,9 +1708,9 @@ public class BlockManager implements BlockStatsMXBean {
         + "pendingReplications", block);
 
     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
-    // remove from neededReplications
+    // remove from neededReconstruction
     if(numEffectiveReplicas + targets.length >= requiredReplication) {
-      neededReplications.remove(block, priority);
+      neededReconstruction.remove(block, priority);
     }
     return true;
   }
@@ -1852,7 +1855,7 @@ public class BlockManager implements BlockStatsMXBean {
         continue;
       }
 
-      if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY 
+      if (priority != LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY
           && !node.isDecommissionInProgress() 
           && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams) {
         continue; // already reached replication limit
@@ -1905,9 +1908,10 @@ public class BlockManager implements BlockStatsMXBean {
             continue;
           }
           NumberReplicas num = countNodes(timedOutItems[i]);
-          if (isNeededReplication(bi, num.liveReplicas())) {
-            neededReplications.add(bi, num.liveReplicas(), num.readOnlyReplicas(),
-                num.decommissionedAndDecommissioning(), getReplication(bi));
+          if (isNeededReconstruction(bi, num.liveReplicas())) {
+            neededReconstruction.add(bi, num.liveReplicas(),
+                num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
+                getReplication(bi));
           }
         }
       } finally {
@@ -2777,7 +2781,7 @@ public class BlockManager implements BlockStatsMXBean {
    * intended for use with initial block report at startup. If not in startup
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * called "immediately" so there is no need to refresh the storedBlock from
-   * blocksMap. Doesn't handle underReplication/overReplication, or worry about
+   * blocksMap. Doesn't handle low redundancy/extra redundancy, or worry about
    * pendingReplications or corruptReplicas, because it's in startup safe mode.
    * Doesn't log every block, because there are typically millions of them.
    * 
@@ -2812,7 +2816,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * Modify (block-->datanode) map. Remove block from set of
-   * needed replications if this takes care of the problem.
+   * needed reconstruction if this takes care of the problem.
    * @return the block that is stored in blocksMap.
    */
   private Block addStoredBlock(final BlockInfo block,
@@ -2890,24 +2894,25 @@ public class BlockManager implements BlockStatsMXBean {
       return storedBlock;
     }
 
-    // do not try to handle over/under-replicated blocks during first safe mode
+    // do not try to handle extra/low redundancy blocks during first safe mode
     if (!isPopulatingReplQueues()) {
       return storedBlock;
     }
 
-    // handle underReplication/overReplication
+    // handle low redundancy/extra redundancy
     short fileReplication = getExpectedReplicaNum(storedBlock);
-    if (!isNeededReplication(storedBlock, numCurrentReplica)) {
-      neededReplications.remove(storedBlock, numCurrentReplica,
+    if (!isNeededReconstruction(storedBlock, numCurrentReplica)) {
+      neededReconstruction.remove(storedBlock, numCurrentReplica,
           num.readOnlyReplicas(),
           num.decommissionedAndDecommissioning(), fileReplication);
     } else {
-      updateNeededReplications(storedBlock, curReplicaDelta, 0);
+      updateNeededReconstructions(storedBlock, curReplicaDelta, 0);
     }
-    if (shouldProcessOverReplicated(num, fileReplication)) {
-      processOverReplicatedBlock(storedBlock, fileReplication, node, delNodeHint);
+    if (shouldProcessExtraRedundancy(num, fileReplication)) {
+      processExtraRedundancyBlock(storedBlock, fileReplication, node,
+          delNodeHint);
     }
-    // If the file replication has reached desired value
+    // If the file redundancy has reached desired value
     // we can remove any corrupt replicas the block may have
     int corruptReplicasCount = corruptReplicas.numCorruptReplicas(storedBlock);
     int numCorruptNodes = num.corruptReplicas();
@@ -2922,7 +2927,7 @@ public class BlockManager implements BlockStatsMXBean {
     return storedBlock;
   }
 
-  private boolean shouldProcessOverReplicated(NumberReplicas num,
+  private boolean shouldProcessExtraRedundancy(NumberReplicas num,
       int expectedNum) {
     final int numCurrent = num.liveReplicas();
     return numCurrent > expectedNum ||
@@ -2972,42 +2977,44 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * For each block in the name-node verify whether it belongs to any file,
-   * over or under replicated. Place it into the respective queue.
+   * extra or low redundancy. Place it into the respective queue.
    */
   public void processMisReplicatedBlocks() {
     assert namesystem.hasWriteLock();
-    stopReplicationInitializer();
-    neededReplications.clear();
-    replicationQueuesInitializer = new Daemon() {
+    stopReconstructionInitializer();
+    neededReconstruction.clear();
+    reconstructionQueuesInitializer = new Daemon() {
 
       @Override
       public void run() {
         try {
           processMisReplicatesAsync();
         } catch (InterruptedException ie) {
-          LOG.info("Interrupted while processing replication queues.");
+          LOG.info("Interrupted while processing reconstruction queues.");
         } catch (Exception e) {
-          LOG.error("Error while processing replication queues async", e);
+          LOG.error("Error while processing reconstruction queues async", e);
         }
       }
     };
-    replicationQueuesInitializer.setName("Replication Queue Initializer");
-    replicationQueuesInitializer.start();
+    reconstructionQueuesInitializer
+        .setName("Reconstruction Queue Initializer");
+    reconstructionQueuesInitializer.start();
   }
 
   /*
-   * Stop the ongoing initialisation of replication queues
+   * Stop the ongoing initialisation of reconstruction queues
    */
-  private void stopReplicationInitializer() {
-    if (replicationQueuesInitializer != null) {
-      replicationQueuesInitializer.interrupt();
+  private void stopReconstructionInitializer() {
+    if (reconstructionQueuesInitializer != null) {
+      reconstructionQueuesInitializer.interrupt();
       try {
-        replicationQueuesInitializer.join();
+        reconstructionQueuesInitializer.join();
       } catch (final InterruptedException e) {
-        LOG.warn("Interrupted while waiting for replicationQueueInitializer. Returning..");
+        LOG.warn("Interrupted while waiting for "
+            + "reconstructionQueueInitializer. Returning..");
         return;
       } finally {
-        replicationQueuesInitializer = null;
+        reconstructionQueuesInitializer = null;
       }
     }
   }
@@ -3025,7 +3032,7 @@ public class BlockManager implements BlockStatsMXBean {
     long startTimeMisReplicatedScan = Time.monotonicNow();
     Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
-    replicationQueuesInitProgress = 0;
+    reconstructionQueuesInitProgress = 0;
     long totalProcessed = 0;
     long sleepDuration =
         Math.max(1, Math.min(numBlocksPerIteration/1000, 10000));
@@ -3067,7 +3074,7 @@ public class BlockManager implements BlockStatsMXBean {
         totalProcessed += processed;
         // there is a possibility that if any of the blocks deleted/added during
         // initialisation, then progress might be different.
-        replicationQueuesInitProgress = Math.min((double) totalProcessed
+        reconstructionQueuesInitProgress = Math.min((double) totalProcessed
             / totalBlocks, 1.0);
 
         if (!blocksItr.hasNext()) {
@@ -3097,12 +3104,12 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Get the progress of the Replication queues initialisation
+   * Get the progress of the reconstruction queues initialisation
    * 
    * @return Returns values between 0 and 1 for the progress.
    */
-  public double getReplicationQueuesInitProgress() {
-    return replicationQueuesInitProgress;
+  public double getReconstructionQueuesInitProgress() {
+    return reconstructionQueuesInitProgress;
   }
 
   /**
@@ -3134,15 +3141,16 @@ public class BlockManager implements BlockStatsMXBean {
     short expectedReplication = getExpectedReplicaNum(block);
     NumberReplicas num = countNodes(block);
     final int numCurrentReplica = num.liveReplicas();
-    // add to under-replicated queue if need to be
-    if (isNeededReplication(block, numCurrentReplica)) {
-      if (neededReplications.add(block, numCurrentReplica, num.readOnlyReplicas(),
-          num.decommissionedAndDecommissioning(), expectedReplication)) {
+    // add to low redundancy queue if need to be
+    if (isNeededReconstruction(block, numCurrentReplica)) {
+      if (neededReconstruction.add(block, numCurrentReplica,
+          num.readOnlyReplicas(), num.decommissionedAndDecommissioning(),
+          expectedReplication)) {
         return MisReplicationResult.UNDER_REPLICATED;
       }
     }
 
-    if (shouldProcessOverReplicated(num, expectedReplication)) {
+    if (shouldProcessExtraRedundancy(num, expectedReplication)) {
       if (num.replicasOnStaleNodes() > 0) {
         // If any of the replicas of this block are on nodes that are
         // considered "stale", then these replicas may in fact have
@@ -3152,8 +3160,8 @@ public class BlockManager implements BlockStatsMXBean {
         return MisReplicationResult.POSTPONE;
       }
       
-      // over-replicated block
-      processOverReplicatedBlock(block, expectedReplication, null, null);
+      // extra redundancy block
+      processExtraRedundancyBlock(block, expectedReplication, null, null);
       return MisReplicationResult.OVER_REPLICATED;
     }
     
@@ -3167,12 +3175,12 @@ public class BlockManager implements BlockStatsMXBean {
       return;
     }
 
-    // update needReplication priority queues
+    // update neededReconstruction priority queues
     b.setReplication(newRepl);
-    updateNeededReplications(b, 0, newRepl - oldRepl);
+    updateNeededReconstructions(b, 0, newRepl - oldRepl);
 
     if (oldRepl > newRepl) {
-      processOverReplicatedBlock(b, newRepl, null, null);
+      processExtraRedundancyBlock(b, newRepl, null, null);
     }
   }
 
@@ -3181,7 +3189,7 @@ public class BlockManager implements BlockStatsMXBean {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  private void processOverReplicatedBlock(final BlockInfo block,
+  private void processExtraRedundancyBlock(final BlockInfo block,
       final short replication, final DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
@@ -3405,7 +3413,7 @@ public class BlockManager implements BlockStatsMXBean {
       //
       if (!storedBlock.isDeleted()) {
         bmSafeMode.decrementSafeBlockCount(storedBlock);
-        updateNeededReplications(storedBlock, -1, 0);
+        updateNeededReconstructions(storedBlock, -1, 0);
       }
 
       excessReplicas.remove(node, storedBlock);
@@ -3748,29 +3756,29 @@ public class BlockManager implements BlockStatsMXBean {
   
   /**
    * On stopping decommission, check if the node has excess replicas.
-   * If there are any excess replicas, call processOverReplicatedBlock().
-   * Process over replicated blocks only when active NN is out of safe mode.
+   * If there are any excess replicas, call processExtraRedundancyBlock().
+   * Process extra redundancy blocks only when active NN is out of safe mode.
    */
-  void processOverReplicatedBlocksOnReCommission(
+  void processExtraRedundancyBlocksOnReCommission(
       final DatanodeDescriptor srcNode) {
     if (!isPopulatingReplQueues()) {
       return;
     }
     final Iterator<BlockInfo> it = srcNode.getBlockIterator();
-    int numOverReplicated = 0;
+    int numExtraRedundancy = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
       int expectedReplication = this.getReplication(block);
       NumberReplicas num = countNodes(block);
-      if (shouldProcessOverReplicated(num, expectedReplication)) {
-        // over-replicated block
-        processOverReplicatedBlock(block, (short) expectedReplication, null,
+      if (shouldProcessExtraRedundancy(num, expectedReplication)) {
+        // extra redundancy block
+        processExtraRedundancyBlock(block, (short) expectedReplication, null,
             null);
-        numOverReplicated++;
+        numExtraRedundancy++;
       }
     }
-    LOG.info("Invalidated " + numOverReplicated + " over-replicated blocks on " +
-        srcNode + " during recommissioning");
+    LOG.info("Invalidated " + numExtraRedundancy
+        + " extra redundancy blocks on " + srcNode + " during recommissioning");
   }
 
   /**
@@ -3789,9 +3797,9 @@ public class BlockManager implements BlockStatsMXBean {
 
     updateState();
     if (pendingReplicationBlocksCount == 0 &&
-        underReplicatedBlocksCount == 0) {
-      LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.",
+        lowRedundancyBlocksCount == 0) {
+      LOG.info("Node {} is dead and there are no low redundancy" +
+          " blocks or blocks pending reconstruction. Safe to decommission.",
           node);
       return true;
     }
@@ -3835,9 +3843,9 @@ public class BlockManager implements BlockStatsMXBean {
     block.setNumBytes(BlockCommand.NO_ACK);
     addToInvalidates(block);
     removeBlockFromMap(block);
-    // Remove the block from pendingReplications and neededReplications
+    // Remove the block from pendingReplications and neededReconstruction
     pendingReplications.remove(block);
-    neededReplications.remove(block, UnderReplicatedBlocks.LEVEL);
+    neededReconstruction.remove(block, LowRedundancyBlocks.LEVEL);
     if (postponedMisreplicatedBlocks.remove(block)) {
       postponedMisreplicatedBlocksCount.decrementAndGet();
     }
@@ -3859,8 +3867,8 @@ public class BlockManager implements BlockStatsMXBean {
         new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
   }
 
-  /** updates a block in under replication queue */
-  private void updateNeededReplications(final BlockInfo block,
+  /** updates a block in needed reconstruction queue. */
+  private void updateNeededReconstructions(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3869,14 +3877,14 @@ public class BlockManager implements BlockStatsMXBean {
       }
       NumberReplicas repl = countNodes(block);
       int curExpectedReplicas = getReplication(block);
-      if (isNeededReplication(block, repl.liveReplicas())) {
-        neededReplications.update(block, repl.liveReplicas(), repl.readOnlyReplicas(),
-            repl.decommissionedAndDecommissioning(), curExpectedReplicas,
-            curReplicasDelta, expectedReplicasDelta);
+      if (isNeededReconstruction(block, repl.liveReplicas())) {
+        neededReconstruction.update(block, repl.liveReplicas(),
+            repl.readOnlyReplicas(), repl.decommissionedAndDecommissioning(),
+            curExpectedReplicas, curReplicasDelta, expectedReplicasDelta);
       } else {
         int oldReplicas = repl.liveReplicas()-curReplicasDelta;
         int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-        neededReplications.remove(block, oldReplicas, repl.readOnlyReplicas(),
+        neededReconstruction.remove(block, oldReplicas, repl.readOnlyReplicas(),
             repl.decommissionedAndDecommissioning(), oldExpectedReplicas);
       }
     } finally {
@@ -3885,10 +3893,10 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Check replication of the blocks in the collection.
-   * If any block is needed replication, insert it into the replication queue.
+   * Check sufficient redundancy of the blocks in the collection. If any block
+   * is needed reconstruction, insert it into the reconstruction queue.
    * Otherwise, if the block is more than the expected replication factor,
-   * process it as an over replicated block.
+   * process it as an extra redundancy block.
    */
   public void checkReplication(BlockCollection bc) {
     for (BlockInfo block : bc.getBlocks()) {
@@ -3896,11 +3904,11 @@ public class BlockManager implements BlockStatsMXBean {
       final NumberReplicas n = countNodes(block);
       final int pending = pendingReplications.getNumReplicas(block);
       if (!hasEnoughEffectiveReplicas(block, n, pending, expected)) {
-        neededReplications.add(block, n.liveReplicas() + pending,
+        neededReconstruction.add(block, n.liveReplicas() + pending,
             n.readOnlyReplicas(),
             n.decommissionedAndDecommissioning(), expected);
-      } else if (shouldProcessOverReplicated(n, expected)) {
-        processOverReplicatedBlock(block, expected, null, null);
+      } else if (shouldProcessExtraRedundancy(n, expected)) {
+        processExtraRedundancyBlock(block, expected, null, null);
       }
     }
   }
@@ -3926,7 +3934,7 @@ public class BlockManager implements BlockStatsMXBean {
     try {
       // blocks should not be replicated or removed if safe mode is on
       if (namesystem.isInSafeMode()) {
-        LOG.debug("In safemode, not computing replication work");
+        LOG.debug("In safemode, not computing reconstruction work");
         return 0;
       }
       try {
@@ -3980,10 +3988,10 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * A block needs replication if the number of replicas is less than expected
-   * or if it does not have enough racks.
+   * A block needs reconstruction if the number of replicas is less than
+   * expected or if it does not have enough racks.
    */
-  boolean isNeededReplication(BlockInfo storedBlock, int current) {
+  boolean isNeededReconstruction(BlockInfo storedBlock, int current) {
     int expected = getExpectedReplicaNum(storedBlock);
     return storedBlock.isComplete()
         && (current < expected || !isPlacementPolicySatisfied(storedBlock));
@@ -3997,12 +4005,12 @@ public class BlockManager implements BlockStatsMXBean {
 
   public long getMissingBlocksCount() {
     // not locking
-    return this.neededReplications.getCorruptBlockSize();
+    return this.neededReconstruction.getCorruptBlockSize();
   }
 
   public long getMissingReplOneBlocksCount() {
     // not locking
-    return this.neededReplications.getCorruptReplOneBlockSize();
+    return this.neededReconstruction.getCorruptReplOneBlockSize();
   }
 
   public BlockInfo addBlockCollection(BlockInfo block,
@@ -4050,8 +4058,8 @@ public class BlockManager implements BlockStatsMXBean {
    * Return an iterator over the set of blocks for which there are no replicas.
    */
   public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
-    return neededReplications.iterator(
-        UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+    return neededReconstruction.iterator(
+        LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
 
   /**
@@ -4070,7 +4078,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   /** @return the size of UnderReplicatedBlocks */
   public int numOfUnderReplicatedBlocks() {
-    return neededReplications.size();
+    return neededReconstruction.size();
   }
 
   /**
@@ -4232,7 +4240,7 @@ public class BlockManager implements BlockStatsMXBean {
    * this NameNode.
    */
   public void clearQueues() {
-    neededReplications.clear();
+    neededReconstruction.clear();
     pendingReplications.clear();
     excessReplicas.clear();
     invalidateBlocks.clear();
@@ -4298,7 +4306,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public void shutdown() {
-    stopReplicationInitializer();
+    stopReconstructionInitializer();
     blocksMap.close();
     MBeans.unregister(mxBeanName);
     mxBeanName = null;

+ 27 - 18
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(", ");
+      }
+    }
+  }
 }

+ 22 - 16
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;
@@ -214,10 +215,10 @@ public class DecommissionManager {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
       // Update DN stats maintained by HeartbeatManager
       hbManager.stopDecommission(node);
-      // Over-replicated blocks will be detected and processed when
+      // extra redundancy blocks will be detected and processed when
       // the dead node comes back and send in its full block report.
       if (node.isAlive()) {
-        blockManager.processOverReplicatedBlocksOnReCommission(node);
+        blockManager.processExtraRedundancyBlocksOnReCommission(node);
       }
       // Remove from tracking in DecommissionManager
       pendingNodes.remove(node);
@@ -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()
@@ -507,9 +513,9 @@ public class DecommissionManager {
         final List<BlockInfo> insufficientList,
         boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
-      int underReplicatedBlocks = 0;
+      int lowRedundancyBlocks = 0;
       int decommissionOnlyReplicas = 0;
-      int underReplicatedInOpenFiles = 0;
+      int lowRedundancyInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
         final BlockInfo block = it.next();
@@ -531,22 +537,22 @@ public class DecommissionManager {
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
 
-        // Schedule under-replicated blocks for replication if not already
+        // Schedule low redundancy blocks for reconstruction if not already
         // pending
-        if (blockManager.isNeededReplication(block, liveReplicas)) {
-          if (!blockManager.neededReplications.contains(block) &&
+        if (blockManager.isNeededReconstruction(block, liveReplicas)) {
+          if (!blockManager.neededReconstruction.contains(block) &&
               blockManager.pendingReplications.getNumReplicas(block) == 0 &&
               blockManager.isPopulatingReplQueues()) {
             // Process these blocks only when active NN is out of safe mode.
-            blockManager.neededReplications.add(block,
+            blockManager.neededReconstruction.add(block,
                 liveReplicas, num.readOnlyReplicas(),
                 num.decommissionedAndDecommissioning(),
                 blockManager.getExpectedReplicaNum(block));
           }
         }
 
-        // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated
+        // Even if the block is without sufficient redundancy,
+        // it doesn't block decommission if has sufficient redundancy
         if (isSufficient(block, bc, num)) {
           if (pruneReliableBlocks) {
             it.remove();
@@ -554,7 +560,7 @@ public class DecommissionManager {
           continue;
         }
 
-        // We've found an insufficiently replicated block.
+        // We've found a block without sufficient redundancy.
         if (insufficientList != null) {
           insufficientList.add(block);
         }
@@ -565,18 +571,18 @@ public class DecommissionManager {
           firstReplicationLog = false;
         }
         // Update various counts
-        underReplicatedBlocks++;
+        lowRedundancyBlocks++;
         if (bc.isUnderConstruction()) {
-          underReplicatedInOpenFiles++;
+          lowRedundancyInOpenFiles++;
         }
         if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
           decommissionOnlyReplicas++;
         }
       }
 
-      datanode.decommissioningStatus.set(underReplicatedBlocks,
+      datanode.decommissioningStatus.set(lowRedundancyBlocks,
           decommissionOnlyReplicas,
-          underReplicatedInOpenFiles);
+          lowRedundancyInOpenFiles);
     }
   }
 

+ 76 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/LowRedundancyBlocks.java

@@ -26,9 +26,9 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 
 /**
- * Keep prioritized queues of under replicated blocks.
- * Blocks have replication priority, with priority {@link #QUEUE_HIGHEST_PRIORITY}
- * indicating the highest priority.
+ * Keep prioritized queues of low redundant blocks.
+ * Blocks have redundancy priority, with priority
+ * {@link #QUEUE_HIGHEST_PRIORITY} indicating the highest priority.
  * </p>
  * Having a prioritised queue allows the {@link BlockManager} to select
  * which blocks to replicate first -it tries to give priority to data
@@ -40,19 +40,19 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
- *   <li>{@link #QUEUE_HIGHEST_PRIORITY}: the blocks that must be replicated
+ *   <li>{@link #QUEUE_HIGHEST_PRIORITY}: the blocks that should be redundant
  *   first. That is blocks with only one copy, or blocks with zero live
  *   copies but a copy in a node being decommissioned. These blocks
  *   are at risk of loss if the disk or server on which they
  *   remain fails.</li>
- *   <li>{@link #QUEUE_VERY_UNDER_REPLICATED}: blocks that are very
+ *   <li>{@link #QUEUE_VERY_LOW_REDUNDANCY}: blocks that are very
  *   under-replicated compared to their expected values. Currently
  *   that means the ratio of the ratio of actual:expected means that
  *   there is <i>less than</i> 1:3.</li>. These blocks may not be at risk,
  *   but they are clearly considered "important".
- *   <li>{@link #QUEUE_UNDER_REPLICATED}: blocks that are also under
+ *   <li>{@link #QUEUE_LOW_REDUNDANCY}: blocks that are also under
  *   replicated, and the ratio of actual:expected is good enough that
- *   they do not need to go into the {@link #QUEUE_VERY_UNDER_REPLICATED}
+ *   they do not need to go into the {@link #QUEUE_VERY_LOW_REDUNDANCY}
  *   queue.</li>
  *   <li>{@link #QUEUE_REPLICAS_BADLY_DISTRIBUTED}: there are as least as
  *   many copies of a block as required, but the blocks are not adequately
@@ -63,15 +63,17 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
  *   blocks that are not corrupt higher priority.</li>
  * </ol>
  */
-class UnderReplicatedBlocks implements Iterable<BlockInfo> {
+class LowRedundancyBlocks implements Iterable<BlockInfo> {
   /** The total number of queues : {@value} */
   static final int LEVEL = 5;
   /** The queue with the highest priority: {@value} */
   static final int QUEUE_HIGHEST_PRIORITY = 0;
   /** The queue for blocks that are way below their expected value : {@value} */
-  static final int QUEUE_VERY_UNDER_REPLICATED = 1;
-  /** The queue for "normally" under-replicated blocks: {@value} */
-  static final int QUEUE_UNDER_REPLICATED = 2;
+  static final int QUEUE_VERY_LOW_REDUNDANCY = 1;
+  /**
+   * The queue for "normally" without sufficient redundancy blocks : {@value}.
+   */
+  static final int QUEUE_LOW_REDUNDANCY = 2;
   /** The queue for blocks that have the right number of replicas,
    * but which the block manager felt were badly distributed: {@value}
    */
@@ -86,7 +88,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   private int corruptReplOneBlocks = 0;
 
   /** Create an object. */
-  UnderReplicatedBlocks() {
+  LowRedundancyBlocks() {
     for (int i = 0; i < LEVEL; i++) {
       priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
     }
@@ -102,7 +104,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
     corruptReplOneBlocks = 0;
   }
 
-  /** Return the total number of under replication blocks */
+  /** Return the total number of insufficient redundancy blocks. */
   synchronized int size() {
     int size = 0;
     for (int i = 0; i < LEVEL; i++) {
@@ -111,8 +113,11 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
     return size;
   }
 
-  /** Return the number of under replication blocks excluding corrupt blocks */
-  synchronized int getUnderReplicatedBlockCount() {
+  /**
+   * Return the number of insufficiently redundant blocks excluding corrupt
+   * blocks.
+   */
+  synchronized int getLowRedundancyBlockCount() {
     int size = 0;
     for (int i = 0; i < LEVEL; i++) {
       if (i != QUEUE_WITH_CORRUPT_BLOCKS) {
@@ -132,7 +137,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
     return corruptReplOneBlocks;
   }
 
-  /** Check if a block is in the neededReplication queue */
+  /** Check if a block is in the neededReconstruction queue. */
   synchronized boolean contains(BlockInfo block) {
     for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
       if (set.contains(block)) {
@@ -187,12 +192,12 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       // highest priority
       return QUEUE_HIGHEST_PRIORITY;
     } else if ((curReplicas * 3) < expectedReplicas) {
-      //there is less than a third as many blocks as requested;
-      //this is considered very under-replicated
-      return QUEUE_VERY_UNDER_REPLICATED;
+      //can only afford one replica loss
+      //this is considered very insufficiently redundant blocks.
+      return QUEUE_VERY_LOW_REDUNDANCY;
     } else {
-      //add to the normal queue for under replicated blocks
-      return QUEUE_UNDER_REPLICATED;
+      //add to the normal queue for insufficiently redundant blocks
+      return QUEUE_LOW_REDUNDANCY;
     }
   }
 
@@ -208,17 +213,19 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       // highest risk of loss, highest priority
       return QUEUE_HIGHEST_PRIORITY;
     } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) {
-      // there is less than a third as many blocks as requested;
-      // this is considered very under-replicated
-      return QUEUE_VERY_UNDER_REPLICATED;
+      // can only afford one replica loss
+      // this is considered very insufficiently redundant blocks.
+      return QUEUE_VERY_LOW_REDUNDANCY;
     } else {
-      // add to the normal queue for under replicated blocks
-      return QUEUE_UNDER_REPLICATED;
+      // add to the normal queue for insufficiently redundant blocks.
+      return QUEUE_LOW_REDUNDANCY;
     }
   }
 
-  /** add a block to a under replication queue according to its priority
-   * @param block a under replication block
+  /**
+   * Add a block to insufficiently redundant queue according to its priority.
+   *
+   * @param block a low redundancy block
    * @param curReplicas current number of replicas of the block
    * @param decomissionedReplicas the number of decommissioned replicas
    * @param expectedReplicas expected number of replicas of the block
@@ -238,17 +245,17 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
         corruptReplOneBlocks++;
       }
       NameNode.blockStateChangeLog.debug(
-          "BLOCK* NameSystem.UnderReplicationBlock.add: {}"
-              + " has only {} replicas and need {} replicas so is added to" +
-              " neededReplications at priority level {}", block, curReplicas,
-          expectedReplicas, priLevel);
+          "BLOCK* NameSystem.LowRedundancyBlock.add: {}"
+              + " has only {} replicas and need {} replicas so is added to"
+              + " neededReconstructions at priority level {}",
+          block, curReplicas, expectedReplicas, priLevel);
 
       return true;
     }
     return false;
   }
 
-  /** remove a block from a under replication queue */
+  /** Remove a block from a low redundancy queue. */
   synchronized boolean remove(BlockInfo block,
                               int oldReplicas,
                               int oldReadOnlyReplicas,
@@ -269,7 +276,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   }
 
   /**
-   * Remove a block from the under replication queues.
+   * Remove a block from the low redundancy queues.
    *
    * The priLevel parameter is a hint of which queue to query
    * first: if negative or &gt;= {@link #LEVEL} this shortcutting
@@ -281,14 +288,16 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
    * <i>Warning:</i> This is not a synchronized method.
    * @param block block to remove
    * @param priLevel expected privilege level
-   * @return true if the block was found and removed from one of the priority queues
+   * @return true if the block was found and removed from one of the priority
+   *         queues
    */
   boolean remove(BlockInfo block, int priLevel) {
     if(priLevel >= 0 && priLevel < LEVEL
         && priorityQueues.get(priLevel).remove(block)) {
       NameNode.blockStateChangeLog.debug(
-        "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block {}" +
-            " from priority queue {}", block, priLevel);
+          "BLOCK* NameSystem.LowRedundancyBlock.remove: Removing block {}"
+              + " from priority queue {}",
+          block, priLevel);
       return true;
     } else {
       // Try to remove the block from all queues if the block was
@@ -296,7 +305,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       for (int i = 0; i < LEVEL; i++) {
         if (i != priLevel && priorityQueues.get(i).remove(block)) {
           NameNode.blockStateChangeLog.debug(
-              "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block" +
+              "BLOCK* NameSystem.LowRedundancyBlock.remove: Removing block" +
                   " {} from priority queue {}", block, i);
           return true;
         }
@@ -314,12 +323,13 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
    * to add it to that queue. This ensures that the block will be
    * in its expected priority queue (and only that queue) by the end of the
    * method call.
-   * @param block a under replicated block
+   * @param block a low redundancy block
    * @param curReplicas current number of replicas of the block
    * @param decommissionedReplicas  the number of decommissioned replicas
    * @param curExpectedReplicas expected number of replicas of the block
    * @param curReplicasDelta the change in the replicate count from before
-   * @param expectedReplicasDelta the change in the expected replica count from before
+   * @param expectedReplicasDelta the change in the expected replica count
+   *        from before
    */
   synchronized void update(BlockInfo block, int curReplicas,
                            int readOnlyReplicas, int decommissionedReplicas,
@@ -332,7 +342,7 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
     int oldPri = getPriority(block, oldReplicas, readOnlyReplicas,
         decommissionedReplicas, oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
+      NameNode.stateChangeLog.debug("LowRedundancyBlocks.update " +
         block +
         " curReplicas " + curReplicas +
         " curExpectedReplicas " + curExpectedReplicas +
@@ -346,10 +356,10 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
     }
     if(priorityQueues.get(curPri).add(block)) {
       NameNode.blockStateChangeLog.debug(
-          "BLOCK* NameSystem.UnderReplicationBlock.update: {} has only {} " +
-              "replicas and needs {} replicas so is added to " +
-              "neededReplications at priority level {}", block, curReplicas,
-          curExpectedReplicas, curPri);
+          "BLOCK* NameSystem.LowRedundancyBlock.update: {} has only {} "
+              + "replicas and needs {} replicas so is added to "
+              + "neededReconstructions at priority level {}",
+          block, curReplicas, curExpectedReplicas, curPri);
 
     }
     if (oldPri != curPri || expectedReplicasDelta != 0) {
@@ -365,25 +375,25 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       }
     }
   }
-  
+
   /**
-   * Get a list of block lists to be replicated. The index of block lists
-   * represents its replication priority. Iterates each block list in priority
-   * order beginning with the highest priority list. Iterators use a bookmark to
-   * resume where the previous iteration stopped. Returns when the block count
-   * is met or iteration reaches the end of the lowest priority list, in which
-   * case bookmarks for each block list are reset to the heads of their
-   * respective lists.
+   * Get a list of block lists without sufficient redundancy. The index of
+   * block lists represents its replication priority. Iterates each block list
+   * in priority order beginning with the highest priority list. Iterators use
+   * a bookmark to resume where the previous iteration stopped. Returns when
+   * the block count is met or iteration reaches the end of the lowest priority
+   * list, in which case bookmarks for each block list are reset to the heads
+   * of their respective lists.
    *
-   * @param blocksToProcess - number of blocks to fetch from underReplicated
+   * @param blocksToProcess - number of blocks to fetch from low redundancy
    *          blocks.
-   * @return Return a list of block lists to be replicated. The block list index
-   *         represents its replication priority.
+   * @return Return a list of block lists to be replicated. The block list
+   *         index represents its redundancy priority.
    */
-  synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
+  synchronized List<List<BlockInfo>> chooseLowRedundancyBlocks(
       int blocksToProcess) {
-    final List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
-    
+    final List<List<BlockInfo>> blocksToReconstruct = new ArrayList<>(LEVEL);
+
     int count = 0;
     int priority = 0;
     for (; count < blocksToProcess && priority < LEVEL; priority++) {
@@ -392,11 +402,11 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
         continue;
       }
 
-      // Go through all blocks that need replications with current priority.
-      // Set the iterator to the first unprocessed block at this priority level.
+      // Go through all blocks that need reconstructions with current priority.
+      // Set the iterator to the first unprocessed block at this priority level
       final Iterator<BlockInfo> i = priorityQueues.get(priority).getBookmark();
       final List<BlockInfo> blocks = new LinkedList<>();
-      blocksToReplicate.add(blocks);
+      blocksToReconstruct.add(blocks);
       // Loop through all remaining blocks in the list.
       for(; count < blocksToProcess && i.hasNext(); count++) {
         blocks.add(i.next());
@@ -410,15 +420,15 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       }
     }
 
-    return blocksToReplicate;
+    return blocksToReconstruct;
   }
 
-  /** returns an iterator of all blocks in a given priority queue */
+  /** Returns an iterator of all blocks in a given priority queue. */
   synchronized Iterator<BlockInfo> iterator(int level) {
     return priorityQueues.get(level).iterator();
   }
 
-  /** return an iterator of all the under replication blocks */
+  /** Return an iterator of all the low redundancy blocks. */
   @Override
   public synchronized Iterator<BlockInfo> iterator() {
     final Iterator<LightWeightLinkedSet<BlockInfo>> q = priorityQueues.iterator();
@@ -445,4 +455,4 @@ class UnderReplicatedBlocks implements Iterable<BlockInfo> {
       }
     };
   }
-}
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiverServer.java

@@ -75,7 +75,7 @@ class DataXceiverServer implements Runnable {
     private BlockBalanceThrottler(long bandwidth, int maxThreads) {
       super(bandwidth);
       this.maxThreads.set(maxThreads);
-      LOG.info("Balancing bandwith is " + bandwidth + " bytes/s");
+      LOG.info("Balancing bandwidth is " + bandwidth + " bytes/s");
       LOG.info("Number threads for balancing is " + maxThreads);
     }
 

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -183,6 +183,13 @@ public class ReplicaInPipeline extends ReplicaInfo
     this.writer = writer;
   }
   
+  public void interruptThread() {
+    if (writer != null && writer != Thread.currentThread() 
+        && writer.isAlive()) {
+      this.writer.interrupt();
+    }
+  }
+
   @Override  // Object
   public boolean equals(Object o) {
     return super.equals(o);

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -3112,5 +3112,18 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public void setTimer(Timer newTimer) {
     this.timer = newTimer;
   }
+
+  synchronized void stopAllDataxceiverThreads(FsVolumeImpl volume) {
+    for (String blockPoolId : volumeMap.getBlockPoolList()) {
+      Collection<ReplicaInfo> replicas = volumeMap.replicas(blockPoolId);
+      for (ReplicaInfo replicaInfo : replicas) {
+        if (replicaInfo instanceof ReplicaInPipeline
+            && replicaInfo.getVolume().equals(volume)) {
+          ReplicaInPipeline replicaInPipeline = (ReplicaInPipeline) replicaInfo;
+          replicaInPipeline.interruptThread();
+        }
+      }
+    }
+  }
 }
 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -240,6 +240,11 @@ public class FsVolumeImpl implements FsVolumeSpi {
     Preconditions.checkState(reference.getReferenceCount() > 0);
   }
 
+  @VisibleForTesting
+  int getReferenceCount() {
+    return this.reference.getReferenceCount();
+  }
+
   /**
    * Close this volume.
    * @throws IOException if the volume is closed.
@@ -247,6 +252,7 @@ public class FsVolumeImpl implements FsVolumeSpi {
   void setClosed() throws IOException {
     try {
       this.reference.setClosed();
+      dataset.stopAllDataxceiverThreads(this);
     } catch (ClosedChannelException e) {
       throw new IOException("The volume has already closed.", e);
     }

+ 17 - 1
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 {

+ 4 - 0
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);

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -383,6 +383,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.edits.dir.required</name>
+  <value></value>
+  <description>This should be a subset of dfs.namenode.edits.dir,
+      to ensure that the transaction (edits) file
+      in these places is always up-to-date.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.shared.edits.dir</name>
   <value></value>

+ 17 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -281,21 +281,6 @@
       dust.render('explorer', base.push(d), function(err, out) {
         $('#panel').html(out);
 
-        $('#table-explorer').dataTable( {
-          'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
-          'columns': [
-            {'searchable': false }, //Permissions
-            null, //Owner
-            null, //Group
-            { 'searchable': false, 'render': func_size_render}, //Size
-            { 'searchable': false, 'render': func_time_render}, //Last Modified
-            { 'searchable': false }, //Replication
-            null, //Block Size
-            null, //Name
-            { 'sortable' : false } //Trash
-          ],
-          "deferRender": true
-        });
 
         $('.explorer-browse-links').click(function() {
           var type = $(this).attr('inode-type');
@@ -324,7 +309,23 @@
           var inode_name = $(this).closest('tr').attr('inode-path');
           var absolute_file_path = append_path(current_directory, inode_name);
           delete_path(inode_name, absolute_file_path);
-        })
+        });
+          
+          $('#table-explorer').dataTable( {
+              'lengthMenu': [ [25, 50, 100, -1], [25, 50, 100, "All"] ],
+              'columns': [
+                  {'searchable': false }, //Permissions
+                  null, //Owner
+                  null, //Group
+                  { 'searchable': false, 'render': func_size_render}, //Size
+                  { 'searchable': false, 'render': func_time_render}, //Last Modified
+                  { 'searchable': false }, //Replication
+                  null, //Block Size
+                  null, //Name
+                  { 'sortable' : false } //Trash
+              ],
+              "deferRender": true
+          });
       });
     }).error(network_error_handler(url));
   }

+ 103 - 56
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -19,6 +19,7 @@ HDFS Commands Guide
 * [User Commands](#User_Commands)
     * [classpath](#classpath)
     * [dfs](#dfs)
+    * [envvars](#envvars)
     * [fetchdt](#fetchdt)
     * [fsck](#fsck)
     * [getconf](#getconf)
@@ -88,6 +89,12 @@ Usage: `hdfs dfs [COMMAND [COMMAND_OPTIONS]]`
 
 Run a filesystem command on the file system supported in Hadoop. The various COMMAND\_OPTIONS can be found at [File System Shell Guide](../hadoop-common/FileSystemShell.html).
 
+### `envvars`
+
+Usage: `hdfs envvars`
+
+display computed Hadoop environment variables.
+
 ### `fetchdt`
 
 Usage: `hdfs fetchdt <opts> <token_file_path> `
@@ -182,7 +189,8 @@ Usage: `hdfs jmxget [-localVM ConnectorURL | -port port | -server mbeanserver |
 | `-help` | print help |
 | `-localVM` ConnectorURL | connect to the VM on the same machine |
 | `-port` *mbean server port* | specify mbean server port, if missing it will try to connect to MBean Server in the same VM |
-| `-service` | specify jmx service, either DataNode or NameNode, the default |
+| `-server` | specify mbean server (localhost by default) |
+| `-service` NameNode\|DataNode | specify jmx service. NameNode by default. |
 
 Dump JMX information from a service.
 
@@ -203,11 +211,11 @@ Usage: `hdfs oev [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE`
 |:---- |:---- |
 | `-f`,`--fix-txids` | Renumber the transaction IDs in the input, so that there are no gaps or invalid transaction IDs. |
 | `-h`,`--help` | Display usage information and exit |
-| `-r`,`--ecover` | When reading binary edit logs, use recovery mode. This will give you the chance to skip corrupt parts of the edit log. |
+| `-r`,`--recover` | When reading binary edit logs, use recovery mode. This will give you the chance to skip corrupt parts of the edit log. |
 | `-p`,`--processor` *arg* | Select which type of processor to apply against image file, currently supported processors are: binary (native binary format that Hadoop uses), xml (default, XML format), stats (prints statistics about edits file) |
 | `-v`,`--verbose` | More verbose output, prints the input and output filenames, for processors that write to a file, also output to screen. On large image files this will dramatically increase processing time (default is false). |
 
-Hadoop offline edits viewer.
+Hadoop offline edits viewer. See [Offline Edits Viewer Guide](./HdfsEditsViewer.html) for more info.
 
 ### `oiv`
 
@@ -217,17 +225,24 @@ Usage: `hdfs oiv [OPTIONS] -i INPUT_FILE`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-i`,`--inputFile` *arg* | edits file to process, xml (case insensitive) extension means XML format, any other filename means binary format |
+| `-i`\|`--inputFile` *input file* | Specify the input fsimage file (or XML file, if ReverseXML processor is used) to process. |
+
 
 #### Optional command line arguments:
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-h`,`--help` | Display usage information and exit |
-| `-o`,`--outputFile` *arg* | Name of output file. If the specified file exists, it will be overwritten, format of the file is determined by -p option |
-| `-p`,`--processor` *arg* | Select which type of processor to apply against image file, currently supported processors are: binary (native binary format that Hadoop uses), xml (default, XML format), stats (prints statistics about edits file) |
+| `-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. |
+| `-delimiter` *arg* | Delimiting string to use with Delimited processor. |
+| `-t`,`--temp` *temporary dir* | Use temporary dir to cache intermediate result to generate Delimited outputs. If not set, Delimited processor constructs the namespace in memory before outputting text. |
+| `-h`,`--help` | Display the tool usage and help information and exit. |
 
-Hadoop Offline Image Viewer for newer image files.
+
+Hadoop Offline Image Viewer for image files in Hadoop 2.4 or up. See [Offline Image Viewer Guide](./HdfsImageViewer.html) for more info.
 
 ### `oiv_legacy`
 
@@ -235,11 +250,21 @@ Usage: `hdfs oiv_legacy [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| `-h`,`--help` | Display usage information and exit |
-| `-i`,`--inputFile` *arg* | edits file to process, xml (case insensitive) extension means XML format, any other filename means binary format |
-| `-o`,`--outputFile` *arg* | Name of output file. If the specified file exists, it will be overwritten, format of the file is determined by -p option |
+| `-i`,`--inputFile` *input file* | Specify the input fsimage file to process. |
+| `-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. |
 
-Hadoop offline image viewer for older versions of Hadoop.
+#### Optional command line arguments:
+
+| COMMAND\_OPTION | Description |
+|:---- |:---- |
+| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Valid options are Ls (default), XML, Delimited, Indented, and FileDistribution. |
+| `-skipBlocks` | Do not enumerate individual blocks within files. This may save processing time and outfile file space on namespaces with very large files. The Ls processor reads the blocks to correctly determine file sizes and ignores this option. |
+| `-printToScreen` | Pipe output of processor to console as well as specified file. On extremely large namespaces, this may increase processing time by an order of magnitude. |
+| `-delimiter` *arg* | When used in conjunction with the Delimited processor, replaces the default tab delimiter with the string specified by *arg*. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |
+
+
+Hadoop offline image viewer for older versions of Hadoop. See [oiv\_legacy Command](./HdfsImageViewer.html#oiv_legacy_Command) for more info.
 
 ### `snapshotDiff`
 
@@ -263,13 +288,14 @@ Commands useful for administrators of a hadoop cluster.
 Usage:
 
         hdfs balancer
-              [-threshold <threshold>]
               [-policy <policy>]
+              [-threshold <threshold>]
               [-exclude [-f <hosts-file> | <comma-separated list of hosts>]]
               [-include [-f <hosts-file> | <comma-separated list of hosts>]]
               [-source [-f <hosts-file> | <comma-separated list of hosts>]]
               [-blockpools <comma-separated list of blockpool ids>]
               [-idleiterations <idleiterations>]
+              [-runDuringUpgrade]
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
@@ -280,6 +306,8 @@ Usage:
 | `-source -f` \<hosts-file\> \| \<comma-separated list of hosts\> | Pick only the specified datanodes as source nodes. |
 | `-blockpools` \<comma-separated list of blockpool ids\> | The balancer will only run on blockpools included in this list. |
 | `-idleiterations` \<iterations\> | Maximum number of idle iterations before exit. This overwrites the default idleiterations(5). |
+| `-runDuringUpgrade` | Whether to run the balancer during an ongoing HDFS upgrade. This is usually not desired since it will not affect used space on over-utilized machines. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |
 
 Runs a cluster balancing utility. An administrator can simply press Ctrl-C to stop the rebalancing process. See [Balancer](./HdfsUserGuide.html#Balancer) for more details.
 
@@ -289,7 +317,18 @@ Besides the above command options, a pinning feature is introduced starting from
 
 ### `cacheadmin`
 
-Usage: `hdfs cacheadmin -addDirective -path <path> -pool <pool-name> [-force] [-replication <replication>] [-ttl <time-to-live>]`
+Usage:
+
+    hdfs cacheadmin [-addDirective -path <path> -pool <pool-name> [-force] [-replication <replication>] [-ttl <time-to-live>]]
+    hdfs cacheadmin [-modifyDirective -id <id> [-path <path>] [-force] [-replication <replication>] [-pool <pool-name>] [-ttl <time-to-live>]]
+    hdfs cacheadmin [-listDirectives [-stats] [-path <path>] [-pool <pool>] [-id <id>]
+    hdfs cacheadmin [-removeDirective <id>]
+    hdfs cacheadmin [-removeDirectives -path <path>]
+    hdfs cacheadmin [-addPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-limit <limit>] [-maxTtl <maxTtl>]
+    hdfs cacheadmin [-modifyPool <name> [-owner <owner>] [-group <group>] [-mode <mode>] [-limit <limit>] [-maxTtl <maxTtl>]]
+    hdfs cacheadmin [-removePool <name>]
+    hdfs cacheadmin [-listPools [-stats] [<name>]]
+    hdfs cacheadmin [-help <command-name>]
 
 See the [HDFS Cache Administration Documentation](./CentralizedCacheManagement.html#cacheadmin_command-line_interface) for more information.
 
@@ -298,8 +337,8 @@ See the [HDFS Cache Administration Documentation](./CentralizedCacheManagement.h
 Usage:
 
       hdfs crypto -createZone -keyName <keyName> -path <path>
-      hdfs crypto -help <command-name>
       hdfs crypto -listZones
+      hdfs crypto -help <command-name>
 
 See the [HDFS Transparent Encryption Documentation](./TransparentEncryption.html#crypto_command-line_interface) for more information.
 
@@ -319,44 +358,43 @@ Runs a HDFS datanode.
 
 Usage:
 
-        hdfs dfsadmin [GENERIC_OPTIONS]
-              [-report [-live] [-dead] [-decommissioning]]
-              [-safemode enter | leave | get | wait | forceExit]
-              [-saveNamespace]
-              [-rollEdits]
-              [-restoreFailedStorage true |false |check]
-              [-refreshNodes]
-              [-setQuota <quota> <dirname>...<dirname>]
-              [-clrQuota <dirname>...<dirname>]
-              [-setSpaceQuota <quota> [-storageType <storagetype>] <dirname>...<dirname>]
-              [-clrSpaceQuota [-storageType <storagetype>] <dirname>...<dirname>]
-              [-finalizeUpgrade]
-              [-rollingUpgrade [<query> |<prepare> |<finalize>]]
-              [-metasave filename]
-              [-refreshServiceAcl]
-              [-refreshUserToGroupsMappings]
-              [-refreshSuperUserGroupsConfiguration]
-              [-refreshCallQueue]
-              [-refresh <host:ipc_port> <key> [arg1..argn]]
-              [-reconfig <datanode |...> <host:ipc_port> <start |status>]
-              [-printTopology]
-              [-refreshNamenodes datanodehost:port]
-              [-deleteBlockPool datanode-host:port blockpoolId [force]]
-              [-setBalancerBandwidth <bandwidth in bytes per second>]
-              [-getBalancerBandwidth <datanode_host:ipc_port>]
-              [-allowSnapshot <snapshotDir>]
-              [-disallowSnapshot <snapshotDir>]
-              [-fetchImage <local directory>]
-              [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
-              [-getDatanodeInfo <datanode_host:ipc_port>]
-              [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
-              [-help [cmd]]
+        hdfs dfsadmin [-report [-live] [-dead] [-decommissioning]]
+        hdfs dfsadmin [-safemode enter | leave | get | wait | forceExit]
+        hdfs dfsadmin [-saveNamespace [-beforeShutdown]]
+        hdfs dfsadmin [-rollEdits]
+        hdfs dfsadmin [-restoreFailedStorage true |false |check]
+        hdfs dfsadmin [-refreshNodes]
+        hdfs dfsadmin [-setQuota <quota> <dirname>...<dirname>]
+        hdfs dfsadmin [-clrQuota <dirname>...<dirname>]
+        hdfs dfsadmin [-setSpaceQuota <quota> [-storageType <storagetype>] <dirname>...<dirname>]
+        hdfs dfsadmin [-clrSpaceQuota [-storageType <storagetype>] <dirname>...<dirname>]
+        hdfs dfsadmin [-finalizeUpgrade]
+        hdfs dfsadmin [-rollingUpgrade [<query> |<prepare> |<finalize>]]
+        hdfs dfsadmin [-refreshServiceAcl]
+        hdfs dfsadmin [-refreshUserToGroupsMappings]
+        hdfs dfsadmin [-refreshSuperUserGroupsConfiguration]
+        hdfs dfsadmin [-refreshCallQueue]
+        hdfs dfsadmin [-refresh <host:ipc_port> <key> [arg1..argn]]
+        hdfs dfsadmin [-reconfig <namenode|datanode> <host:ipc_port> <start |status |properties>]
+        hdfs dfsadmin [-printTopology]
+        hdfs dfsadmin [-refreshNamenodes datanodehost:port]
+        hdfs dfsadmin [-deleteBlockPool datanode-host:port blockpoolId [force]]
+        hdfs dfsadmin [-setBalancerBandwidth <bandwidth in bytes per second>]
+        hdfs dfsadmin [-getBalancerBandwidth <datanode_host:ipc_port>]
+        hdfs dfsadmin [-fetchImage <local directory>]
+        hdfs dfsadmin [-allowSnapshot <snapshotDir>]
+        hdfs dfsadmin [-disallowSnapshot <snapshotDir>]
+        hdfs dfsadmin [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
+        hdfs dfsadmin [-getDatanodeInfo <datanode_host:ipc_port>]
+        hdfs dfsadmin [-metasave filename]
+        hdfs dfsadmin [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
+        hdfs dfsadmin [-help [cmd]]
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
 | `-report` `[-live]` `[-dead]` `[-decommissioning]` | Reports basic filesystem information and statistics, The dfs usage can be different from "du" usage, because it measures raw space used by replication, checksums, snapshots and etc. on all the DNs. Optional flags may be used to filter the list of displayed DataNodes. |
 | `-safemode` enter\|leave\|get\|wait\|forceExit | Safe mode maintenance command. Safe mode is a Namenode state in which it <br/>1. does not accept changes to the name space (read-only) <br/>2. does not replicate or delete blocks. <br/>Safe mode is entered automatically at Namenode startup, and leaves safe mode automatically when the configured minimum percentage of blocks satisfies the minimum replication condition. If Namenode detects any anomaly then it will linger in safe mode till that issue is resolved. If that anomaly is the consequence of a deliberate action, then administrator can use -safemode forceExit to exit safe mode. The cases where forceExit may be required are<br/> 1. Namenode metadata is not consistent. If Namenode detects that metadata has been modified out of band and can cause data loss, then Namenode will enter forceExit state. At that point user can either restart Namenode with correct metadata files or forceExit (if data loss is acceptable).<br/>2. Rollback causes metadata to be replaced and rarely it can trigger safe mode forceExit state in Namenode. In that case you may proceed by issuing -safemode forceExit.<br/> Safe mode can also be entered manually, but then it can only be turned off manually as well. |
-| `-saveNamespace` | Save current namespace into storage directories and reset edits log. Requires safe mode. |
+| `-saveNamespace` `\[-beforeShutdown\]` | Save current namespace into storage directories and reset edits log. Requires safe mode. If the "beforeShutdown" option is given, the NameNode does a checkpoint if and only if no checkpoint has been done during a time window (a configurable number of checkpoint periods). This is usually used before shutting down the NameNode to prevent potential fsimage/editlog corruption. |
 | `-rollEdits` | Rolls the edit log on the active NameNode. |
 | `-restoreFailedStorage` true\|false\|check | This option will turn on/off automatic attempt to restore failed storage replicas. If a failed storage becomes available again the system will attempt to restore edits and/or fsimage during checkpoint. 'check' option will return current setting. |
 | `-refreshNodes` | Re-read the hosts and exclude files to update the set of Datanodes that are allowed to connect to the Namenode and those that should be decommissioned or recommissioned. |
@@ -366,23 +404,23 @@ Usage:
 | `-clrSpaceQuota` `[-storageType <storagetype>]` \<dirname\>...\<dirname\> | See [HDFS Quotas Guide](../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands) for the detail. |
 | `-finalizeUpgrade` | Finalize upgrade of HDFS. Datanodes delete their previous version working directories, followed by Namenode doing the same. This completes the upgrade process. |
 | `-rollingUpgrade` [\<query\>\|\<prepare\>\|\<finalize\>] | See [Rolling Upgrade document](../hadoop-hdfs/HdfsRollingUpgrade.html#dfsadmin_-rollingUpgrade) for the detail. |
-| `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-refreshServiceAcl` | Reload the service-level authorization policy file. |
 | `-refreshUserToGroupsMappings` | Refresh user-to-groups mappings. |
 | `-refreshSuperUserGroupsConfiguration` | Refresh superuser proxy groups mappings |
 | `-refreshCallQueue` | Reload the call queue from config. |
 | `-refresh` \<host:ipc\_port\> \<key\> [arg1..argn] | Triggers a runtime-refresh of the resource specified by \<key\> on \<host:ipc\_port\>. All other args after are sent to the host. |
-| `-reconfig` \<datanode \|...\> \<host:ipc\_port\> \<start\|status\> | Start reconfiguration or get the status of an ongoing reconfiguration. The second parameter specifies the node type. Currently, only reloading DataNode's configuration is supported. |
+| `-reconfig` \<datanode \|namenode\> \<host:ipc\_port\> \<start\|status\|properties\> | Starts reconfiguration or gets the status of an ongoing reconfiguration, or gets a list of reconfigurable properties. The second parameter specifies the node type. |
 | `-printTopology` | Print a tree of the racks and their nodes as reported by the Namenode |
 | `-refreshNamenodes` datanodehost:port | For the given datanode, reloads the configuration files, stops serving the removed block-pools and starts serving new block-pools. |
 | `-deleteBlockPool` datanode-host:port blockpoolId [force] | If force is passed, block pool directory for the given blockpool id on the given datanode is deleted along with its contents, otherwise the directory is deleted only if it is empty. The command will fail if datanode is still serving the block pool. Refer to refreshNamenodes to shutdown a block pool service on a datanode. |
 | `-setBalancerBandwidth` \<bandwidth in bytes per second\> | Changes the network bandwidth used by each datanode during HDFS block balancing. \<bandwidth\> is the maximum number of bytes per second that will be used by each datanode. This value overrides the dfs.balance.bandwidthPerSec parameter. NOTE: The new value is not persistent on the DataNode. |
 | `-getBalancerBandwidth` \<datanode\_host:ipc\_port\> | Get the network bandwidth(in bytes per second) for the given datanode. This is the maximum network bandwidth used by the datanode during HDFS block balancing.|
+| `-fetchImage` \<local directory\> | Downloads the most recent fsimage from the NameNode and saves it in the specified local directory. |
 | `-allowSnapshot` \<snapshotDir\> | Allowing snapshots of a directory to be created. If the operation completes successfully, the directory becomes snapshottable. See the [HDFS Snapshot Documentation](./HdfsSnapshots.html) for more information. |
 | `-disallowSnapshot` \<snapshotDir\> | Disallowing snapshots of a directory to be created. All snapshots of the directory must be deleted before disallowing snapshots. See the [HDFS Snapshot Documentation](./HdfsSnapshots.html) for more information. |
-| `-fetchImage` \<local directory\> | Downloads the most recent fsimage from the NameNode and saves it in the specified local directory. |
 | `-shutdownDatanode` \<datanode\_host:ipc\_port\> [upgrade] | Submit a shutdown request for the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-shutdownDatanode) for the detail. |
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
+| `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
@@ -411,12 +449,13 @@ Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Ad
 
 Usage:
 
-        hdfs haadmin -checkHealth <serviceId>
+        hdfs haadmin -transitionToActive <serviceId> [--forceactive]
+        hdfs haadmin -transitionToStandby <serviceId>
         hdfs haadmin -failover [--forcefence] [--forceactive] <serviceId> <serviceId>
         hdfs haadmin -getServiceState <serviceId>
+        hdfs haadmin -checkHealth <serviceId>
         hdfs haadmin -help <command>
-        hdfs haadmin -transitionToActive <serviceId> [--forceactive]
-        hdfs haadmin -transitionToStandby <serviceId>
+
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
@@ -425,6 +464,7 @@ Usage:
 | `-getServiceState` | determine whether the given NameNode is Active or Standby |
 | `-transitionToActive` | transition the state of the given NameNode to Active (Warning: No fencing is done) |
 | `-transitionToStandby` | transition the state of the given NameNode to Standby (Warning: No fencing is done) |
+| `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 See [HDFS HA with NFS](./HDFSHighAvailabilityWithNFS.html#Administrative_commands) or [HDFS HA with QJM](./HDFSHighAvailabilityWithQJM.html#Administrative_commands) for more information on this command.
 
@@ -509,9 +549,16 @@ Runs the HDFS secondary namenode. See [Secondary Namenode](./HdfsUserGuide.html#
 
 ### `storagepolicies`
 
-Usage: `hdfs storagepolicies`
+Usage:
+
+      hdfs storagepolicies
+          [-listPolicies]
+          [-setStoragePolicy -path <path> -policy <policy>]
+          [-getStoragePolicy -path <path>]
+          [-unsetStoragePolicy -path <path>]
+          [-help <command-name>]
 
-Lists out all storage policies. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.
+Lists out all/Gets/sets/unsets storage policies. See the [HDFS Storage Policy Documentation](./ArchivalStorage.html) for more information.
 
 ### `zkfc`
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

@@ -74,7 +74,7 @@ public class BlockManagerTestUtil {
       final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
           bm.countNodes(storedBlock).liveReplicas(),
-          bm.neededReplications.contains(storedBlock) ? 1 : 0};
+          bm.neededReconstruction.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -397,20 +397,20 @@ public class TestBlockManager {
     addNodes(nodes);
     List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);
     for (int i = 0; i < NUM_TEST_ITERS; i++) {
-      doTestSingleRackClusterIsSufficientlyReplicated(i, origNodes);
+      doTestSingleRackClusterHasSufficientRedundancy(i, origNodes);
     }
   }
   
-  private void doTestSingleRackClusterIsSufficientlyReplicated(int testIndex,
+  private void doTestSingleRackClusterHasSufficientRedundancy(int testIndex,
       List<DatanodeDescriptor> origNodes)
       throws Exception {
     assertEquals(0, bm.numOfUnderReplicatedBlocks());
     BlockInfo block = addBlockOnNodes(testIndex, origNodes);
-    assertFalse(bm.isNeededReplication(block, bm.countLiveNodes(block)));
+    assertFalse(bm.isNeededReconstruction(block, bm.countLiveNodes(block)));
   }
   
   @Test(timeout = 60000)
-  public void testNeededReplicationWhileAppending() throws IOException {
+  public void testNeededReconstructionWhileAppending() throws IOException {
     Configuration conf = new HdfsConfiguration();
     String src = "/test-file";
     Path file = new Path(src);
@@ -449,7 +449,7 @@ public class TestBlockManager {
         namenode.updatePipeline(clientName, oldBlock, newBlock,
             newLocatedBlock.getLocations(), newLocatedBlock.getStorageIDs());
         BlockInfo bi = bm.getStoredBlock(newBlock.getLocalBlock());
-        assertFalse(bm.isNeededReplication(bi, bm.countLiveNodes(bi)));
+        assertFalse(bm.isNeededReconstruction(bi, bm.countLiveNodes(bi)));
       } finally {
         IOUtils.closeStream(out);
       }
@@ -601,7 +601,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
 
     assertEquals("Does not choose a source node for a less-than-highest-priority"
             + " replication since all available source nodes have reached"
@@ -612,7 +612,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).length);
+            LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY).length);
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
@@ -626,7 +626,7 @@ public class TestBlockManager {
             liveNodes,
             new NumberReplicas(),
             new ArrayList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length);
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY).length);
   }
 
   @Test
@@ -652,7 +652,7 @@ public class TestBlockManager {
             cntNodes,
             liveNodes,
             new NumberReplicas(), new LinkedList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)[0]);
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY)[0]);
 
 
     // Increase the replication count to test replication count > hard limit
@@ -666,7 +666,7 @@ public class TestBlockManager {
             cntNodes,
             liveNodes,
             new NumberReplicas(), new LinkedList<Byte>(),
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length);
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY).length);
   }
 
   @Test

+ 24 - 24
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestLowRedundancyBlockQueues.java

@@ -30,7 +30,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
-public class TestUnderReplicatedBlockQueues {
+public class TestLowRedundancyBlockQueues {
 
   private final ErasureCodingPolicy ecPolicy =
       ErasureCodingPolicyManager.getSystemDefaultPolicy();
@@ -52,39 +52,39 @@ public class TestUnderReplicatedBlockQueues {
    */
   @Test
   public void testBlockPriorities() throws Throwable {
-    UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
+    LowRedundancyBlocks queues = new LowRedundancyBlocks();
     BlockInfo block1 = genBlockInfo(1);
     BlockInfo block2 = genBlockInfo(2);
-    BlockInfo block_very_under_replicated = genBlockInfo(3);
+    BlockInfo block_very_low_redundancy = genBlockInfo(3);
     BlockInfo block_corrupt = genBlockInfo(4);
     BlockInfo block_corrupt_repl_one = genBlockInfo(5);
 
     //add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
 
-    assertEquals(1, queues.getUnderReplicatedBlockCount());
+    assertEquals(1, queues.getLowRedundancyBlockCount());
     assertEquals(1, queues.size());
-    assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
+    assertInLevel(queues, block1, LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
     //repeated additions fail
     assertFalse(queues.add(block1, 1, 0, 0, 3));
 
     //add a second block with two replicas
     assertAdded(queues, block2, 2, 0, 3);
-    assertEquals(2, queues.getUnderReplicatedBlockCount());
+    assertEquals(2, queues.getLowRedundancyBlockCount());
     assertEquals(2, queues.size());
-    assertInLevel(queues, block2, UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
+    assertInLevel(queues, block2, LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
     //now try to add a block that is corrupt
     assertAdded(queues, block_corrupt, 0, 0, 3);
     assertEquals(3, queues.size());
-    assertEquals(2, queues.getUnderReplicatedBlockCount());
+    assertEquals(2, queues.getLowRedundancyBlockCount());
     assertEquals(1, queues.getCorruptBlockSize());
     assertInLevel(queues, block_corrupt,
-                  UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+                  LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
 
-    //insert a very under-replicated block
-    assertAdded(queues, block_very_under_replicated, 4, 0, 25);
-    assertInLevel(queues, block_very_under_replicated,
-                  UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
+    //insert a very insufficiently redundancy block
+    assertAdded(queues, block_very_low_redundancy, 4, 0, 25);
+    assertInLevel(queues, block_very_low_redundancy,
+                  LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY);
 
     //insert a corrupt block with replication factor 1
     assertAdded(queues, block_corrupt_repl_one, 0, 0, 1);
@@ -94,7 +94,7 @@ public class TestUnderReplicatedBlockQueues {
     assertEquals(0, queues.getCorruptReplOneBlockSize());
     queues.update(block_corrupt, 0, 0, 0, 1, 0, -2);
     assertEquals(1, queues.getCorruptReplOneBlockSize());
-    queues.update(block_very_under_replicated, 0, 0, 0, 1, -4, -24);
+    queues.update(block_very_low_redundancy, 0, 0, 0, 1, -4, -24);
     assertEquals(2, queues.getCorruptReplOneBlockSize());
   }
 
@@ -110,26 +110,26 @@ public class TestUnderReplicatedBlockQueues {
       throws Throwable {
     int groupSize = dataBlkNum + parityBlkNum;
     long numBytes = ecPolicy.getCellSize() * dataBlkNum;
-    UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
+    LowRedundancyBlocks queues = new LowRedundancyBlocks();
     int numUR = 0;
     int numCorrupt = 0;
 
-    // add under replicated blocks
+    // add low redundancy blocks
     for (int i = 0; dataBlkNum + i < groupSize; i++) {
       BlockInfo block = genStripedBlockInfo(-100 - 100 * i, numBytes);
       assertAdded(queues, block, dataBlkNum + i, 0, groupSize);
       numUR++;
-      assertEquals(numUR, queues.getUnderReplicatedBlockCount());
+      assertEquals(numUR, queues.getLowRedundancyBlockCount());
       assertEquals(numUR + numCorrupt, queues.size());
       if (i == 0) {
         assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY);
+            LowRedundancyBlocks.QUEUE_HIGHEST_PRIORITY);
       } else if (i * 3 < parityBlkNum + 1) {
         assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED);
+            LowRedundancyBlocks.QUEUE_VERY_LOW_REDUNDANCY);
       } else {
         assertInLevel(queues, block,
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED);
+            LowRedundancyBlocks.QUEUE_LOW_REDUNDANCY);
       }
     }
 
@@ -139,13 +139,13 @@ public class TestUnderReplicatedBlockQueues {
     assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize);
     numCorrupt++;
     assertEquals(numUR + numCorrupt, queues.size());
-    assertEquals(numUR, queues.getUnderReplicatedBlockCount());
+    assertEquals(numUR, queues.getLowRedundancyBlockCount());
     assertEquals(numCorrupt, queues.getCorruptBlockSize());
     assertInLevel(queues, block_corrupt,
-        UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
+        LowRedundancyBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
 
-  private void assertAdded(UnderReplicatedBlocks queues,
+  private void assertAdded(LowRedundancyBlocks queues,
                            BlockInfo block,
                            int curReplicas,
                            int decomissionedReplicas,
@@ -167,7 +167,7 @@ public class TestUnderReplicatedBlockQueues {
    * @param block block to look for
    * @param level level to select
    */
-  private void assertInLevel(UnderReplicatedBlocks queues,
+  private void assertInLevel(LowRedundancyBlocks queues,
                              Block block,
                              int level) {
     final Iterator<BlockInfo> bi = queues.iterator(level);

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java

@@ -183,7 +183,7 @@ public class TestPendingReplication {
 
       PendingReplicationBlocks pendingReplications =
           blkManager.pendingReplications;
-      UnderReplicatedBlocks neededReplications = blkManager.neededReplications;
+      LowRedundancyBlocks neededReconstruction = blkManager.neededReconstruction;
       BlocksMap blocksMap = blkManager.blocksMap;
 
       //
@@ -227,9 +227,9 @@ public class TestPendingReplication {
       }
 
       //
-      // Verify that block moves to neededReplications
+      // Verify that block moves to neededReconstruction
       //
-      while (neededReplications.size() == 0) {
+      while (neededReconstruction.size() == 0) {
         try {
           Thread.sleep(100);
         } catch (Exception e) {
@@ -238,14 +238,14 @@ public class TestPendingReplication {
 
       // Verify that the generation stamp we will try to replicate
       // is now 1
-      for (Block b: neededReplications) {
+      for (Block b: neededReconstruction) {
         assertEquals("Generation stamp is 1 ", 1,
             b.getGenerationStamp());
       }
 
-      // Verify size of neededReplications is exactly 1.
-      assertEquals("size of neededReplications is 1 ", 1,
-          neededReplications.size());
+      // Verify size of neededReconstruction is exactly 1.
+      assertEquals("size of neededReconstruction is 1 ", 1,
+          neededReconstruction.size());
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 79 - 79
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -836,12 +836,12 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
         .format(true).build();
     try {
       cluster.waitActive();
-      final UnderReplicatedBlocks neededReplications = cluster.getNameNode()
-          .getNamesystem().getBlockManager().neededReplications;
+      final LowRedundancyBlocks neededReconstruction = cluster.getNameNode()
+          .getNamesystem().getBlockManager().neededReconstruction;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
 
-        neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+        neededReconstruction.add(genBlockInfo(ThreadLocalRandom.current().
             nextLong()), 2, 0, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
@@ -849,7 +849,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+      neededReconstruction.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 1, 0, 0, 3);
 
       // Lets wait for the replication interval
@@ -858,68 +858,68 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
       // Check replication completed successfully. Need not wait till it process
       // all the 100 normal blocks.
       assertFalse("Not able to clear the element from high priority list",
-          neededReplications.iterator(HIGH_PRIORITY).hasNext());
+          neededReconstruction.iterator(HIGH_PRIORITY).hasNext());
     } finally {
       cluster.shutdown();
     }
   }
   
   /**
-   * Test for the ChooseUnderReplicatedBlocks are processed based on priority
+   * Test for the ChooseLowRedundancyBlocks are processed based on priority
    */
   @Test
-  public void testChooseUnderReplicatedBlocks() throws Exception {
-    UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
+  public void testChooseLowRedundancyBlocks() throws Exception {
+    LowRedundancyBlocks lowRedundancyBlocks = new LowRedundancyBlocks();
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 1, 0, 0, 3);
 
-      // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      // Adding QUEUE_VERY_LOW_REDUNDANCY block
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 2, 0, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 6, 0, 0, 6);
 
-      // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      // Adding QUEUE_LOW_REDUNDANCY block
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 5, 0, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+      lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
           nextLong()), 0, 0, 0, 3);
     }
 
-    // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
-    // from
-    // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 6 blocks from lowRedundancyBlocks. Then it should pick 5 blocks
+    // from QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_LOW_REDUNDANCY.
     List<List<BlockInfo>> chosenBlocks =
-        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+        lowRedundancyBlocks.chooseLowRedundancyBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
-    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
-    // QUEUE_VERY_UNDER_REPLICATED, 5 blocks from QUEUE_UNDER_REPLICATED and 1
+    // Choose 10 blocks from lowRedundancyBlocks. Then it should pick 4 blocks
+    // from QUEUE_VERY_LOW_REDUNDANCY, 5 blocks from QUEUE_LOW_REDUNDANCY and 1
     // block from QUEUE_REPLICAS_BADLY_DISTRIBUTED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(10);
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+    lowRedundancyBlocks.add(genBlockInfo(ThreadLocalRandom.current().
         nextLong()), 0, 1, 0, 3);
 
-    // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
-    // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(10);
+    // Choose 10 blocks from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_HIGHEST_PRIORITY, 4 blocks from
+    // QUEUE_REPLICAS_BADLY_DISTRIBUTED
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(10);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 4);
 
     // Since it is reached to end of all lists,
     // should start picking the blocks from start.
-    // Choose 7 blocks from UnderReplicatedBlocks. Then it should pick 6 blocks from
-    // QUEUE_HIGHEST_PRIORITY, 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(7);
+    // Choose 7 blocks from lowRedundancyBlocks. Then it should pick 6 blocks
+    // from QUEUE_HIGHEST_PRIORITY, 1 block from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(7);
     assertTheChosenBlocks(chosenBlocks, 6, 1, 0, 0, 0);
   }
   
@@ -1268,45 +1268,45 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
 
   @Test(timeout = 60000)
   public void testUpdateDoesNotCauseSkippedReplication() {
-    UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
+    LowRedundancyBlocks lowRedundancyBlocks = new LowRedundancyBlocks();
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block3 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_VERY_UNDER_REPLICATED block
+    // Adding QUEUE_VERY_LOW_REDUNDANCY block
     final int block1CurReplicas = 2;
     final int block1ExpectedReplicas = 7;
-    underReplicatedBlocks.add(block1, block1CurReplicas, 0, 0,
+    lowRedundancyBlocks.add(block1, block1CurReplicas, 0, 0,
         block1ExpectedReplicas);
 
-    // Adding QUEUE_VERY_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 2, 0, 0, 7);
+    // Adding QUEUE_VERY_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 2, 0, 0, 7);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block3, 2, 0, 0, 6);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block3, 2, 0, 0, 6);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 1, 0, 0, 0);
 
     // Increasing the replications will move the block down a
     // priority.  This simulates a replica being completed in between checks.
-    underReplicatedBlocks.update(block1, block1CurReplicas+1, 0, 0,
+    lowRedundancyBlocks.update(block1, block1CurReplicas+1, 0, 0,
         block1ExpectedReplicas, 1, 0);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block was moved up a priority and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 1, 0, 0, 0);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 0, 0, 1, 0, 0);
   }
 
@@ -1317,27 +1317,27 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasWriteLock()).thenReturn(true);
     when(mockNS.hasReadLock()).thenReturn(true);
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    // Adding this block collection to the BlockManager, so that when we add the
+    // Adding this block collection to the BlockManager, so that when we add
     // block under construction, the BlockManager will realize the expected
-    // replication has been achieved and remove it from the under-replicated
+    // replication has been achieved and remove it from the low redundancy
     // queue.
     BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
@@ -1353,9 +1353,9 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
         ReplicaState.FINALIZED), storages[0]);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 
@@ -1367,7 +1367,7 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasWriteLock()).thenReturn(true);
 
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     long blkID1 = ThreadLocalRandom.current().nextLong();
     if (blkID1 < 0) {
@@ -1381,17 +1381,17 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     BlockInfo block1 = genBlockInfo(blkID1);
     BlockInfo block2 = genBlockInfo(blkID2);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
     final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
@@ -1425,10 +1425,10 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 
@@ -1439,30 +1439,30 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
     when(mockNS.hasReadLock()).thenReturn(true);
 
     BlockManager bm = new BlockManager(mockNS, false, new HdfsConfiguration());
-    UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
+    LowRedundancyBlocks lowRedundancyBlocks = bm.neededReconstruction;
 
     BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
     BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block1, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block1, 0, 0, 1, 1);
 
-    // Adding QUEUE_UNDER_REPLICATED block
-    underReplicatedBlocks.add(block2, 0, 0, 1, 1);
+    // Adding QUEUE_LOW_REDUNDANCY block
+    lowRedundancyBlocks.add(block2, 0, 0, 1, 1);
 
     List<List<BlockInfo>> chosenBlocks;
 
-    // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    // Choose 1 block from lowRedundancyBlocks. Then it should pick 1 block
+    // from QUEUE_VERY_LOW_REDUNDANCY.
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
     bm.setReplication((short)0, (short)1, block1);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
-    // from QUEUE_VERY_UNDER_REPLICATED.
+    // from QUEUE_VERY_LOW_REDUNDANCY.
     // This block remains and should not be skipped over.
-    chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
+    chosenBlocks = lowRedundancyBlocks.chooseLowRedundancyBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
   }
 

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeUUID.java

@@ -19,17 +19,21 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.Test;
 
+import java.io.File;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
 
 public class TestDataNodeUUID {
 
@@ -62,4 +66,52 @@ public class TestDataNodeUUID {
     // Make sure that we have a valid DataNodeUUID at that point of time.
     assertNotEquals(dn.getDatanodeUuid(), nullString);
   }
+
+  @Test(timeout = 10000)
+  public void testUUIDRegeneration() throws Exception {
+    File baseDir = new File(System.getProperty("test.build.data"));
+    File disk1 = new File(baseDir, "disk1");
+    File disk2 = new File(baseDir, "disk2");
+
+    // Ensure the configured disks do not pre-exist
+    FileUtils.deleteDirectory(disk1);
+    FileUtils.deleteDirectory(disk2);
+
+    MiniDFSCluster cluster = null;
+    HdfsConfiguration conf = new HdfsConfiguration();
+    conf.setStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+            disk1.toURI().toString(),
+            disk2.toURI().toString());
+    try {
+      cluster = new MiniDFSCluster.Builder(conf)
+              .numDataNodes(1)
+              .manageDataDfsDirs(false)
+              .build();
+      cluster.waitActive();
+
+      // Grab the new-cluster UUID as the original one to test against
+      String originalUUID = cluster.getDataNodes().get(0).getDatanodeUuid();
+      // Stop and simulate a DN wipe or unmount-but-root-path condition
+      // on the second disk
+      MiniDFSCluster.DataNodeProperties dn = cluster.stopDataNode(0);
+      FileUtils.deleteDirectory(disk2);
+      assertTrue("Failed to recreate the data directory: " + disk2,
+              disk2.mkdirs());
+
+      // Restart and check if the UUID changed
+      assertTrue("DataNode failed to start up: " + dn,
+              cluster.restartDataNode(dn));
+      // We need to wait until the DN has completed registration
+      while (!cluster.getDataNodes().get(0).isDatanodeFullyStarted()) {
+        Thread.sleep(50);
+      }
+      assertEquals(
+              "DN generated a new UUID despite disk1 having it intact",
+              originalUUID, cluster.getDataNodes().get(0).getDatanodeUuid());
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 66 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -21,14 +21,19 @@ import com.google.common.collect.Lists;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -51,6 +56,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.FakeTimer;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
@@ -622,4 +628,64 @@ public class TestFsDatasetImpl {
     LOG.info("Volumes removed");
     brReceivedLatch.await();
   }
+
+  /**
+   * Tests stopping all the active DataXceiver thread on volume failure event.
+   * @throws Exception
+   */
+  @Test
+  public void testCleanShutdownOfVolume() throws Exception {
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration config = new HdfsConfiguration();
+      config.setLong(
+          DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY, 1000);
+      config.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+      cluster.waitActive();
+      FileSystem fs = cluster.getFileSystem();
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      Path filePath = new Path("test.dat");
+      // Create a file and keep the output stream unclosed.
+      FSDataOutputStream out = fs.create(filePath, (short) 1);
+      out.write(1);
+      out.hflush();
+
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, filePath);
+      FsVolumeImpl volume = (FsVolumeImpl) dataNode.getFSDataset().getVolume(
+          block);
+      File finalizedDir = volume.getFinalizedDir(cluster.getNamesystem()
+          .getBlockPoolId());
+
+      if (finalizedDir.exists()) {
+        // Remove write and execute access so that checkDiskErrorThread detects
+        // this volume is bad.
+        finalizedDir.setExecutable(false);
+        finalizedDir.setWritable(false);
+      }
+      Assert.assertTrue("Reference count for the volume should be greater "
+          + "than 0", volume.getReferenceCount() > 0);
+      // Invoke the synchronous checkDiskError method
+      dataNode.getFSDataset().checkDataDir();
+      // Sleep for 1 second so that datanode can interrupt and cluster clean up
+      Thread.sleep(1000);
+      assertEquals("There are active threads still referencing volume: "
+          + volume.getBasePath(), 0, volume.getReferenceCount());
+      LocatedBlock lb = DFSTestUtil.getAllBlocks(fs, filePath).get(0);
+      DatanodeInfo info = lb.getLocations()[0];
+
+      try {
+        out.close();
+        Assert.fail("This is not a valid code path. "
+            + "out.close should have thrown an exception.");
+      } catch (IOException ioe) {
+        Assert.assertTrue(ioe.getMessage().contains(info.toString()));
+      }
+      finalizedDir.setWritable(true);
+      finalizedDir.setExecutable(true);
+    } finally {
+    cluster.shutdown();
+    }
+  }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java

@@ -152,7 +152,7 @@ public class TestMetaSave {
       line = reader.readLine();
       assertTrue(line.equals("Dead Datanodes: 1"));
       line = reader.readLine();
-      assertTrue(line.equals("Metasave: Blocks waiting for replication: 0"));
+      assertTrue(line.equals("Metasave: Blocks waiting for reconstruction: 0"));
       line = reader.readLine();
       assertTrue(line.equals("Mis-replicated blocks that have been postponed:"));
       line = reader.readLine();

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

@@ -291,6 +291,11 @@ public class TestStandbyCheckpoints {
    */
   @Test(timeout=60000)
   public void testCheckpointCancellationDuringUpload() throws Exception {
+    // Set dfs.namenode.checkpoint.txns differently on the first NN to avoid it
+    // doing checkpoint when it becomes a standby
+    cluster.getConfiguration(0).setInt(
+        DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1000);
+
     // don't compress, we want a big image
     for (int i = 0; i < NUM_NNS; i++) {
       cluster.getConfiguration(i).setBoolean(

+ 34 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ClusterStatus.java

@@ -171,7 +171,40 @@ public class ClusterStatus implements Writable {
       sb.append(blackListReport.replace("\n", ":"));
       return sb.toString();
     }
-    
+
+    @Override
+    public int hashCode() {
+      int result = trackerName != null ? trackerName.hashCode() : 0;
+      result = 31 * result + (reasonForBlackListing != null ?
+          reasonForBlackListing.hashCode() : 0);
+      result = 31 * result + (blackListReport != null ?
+          blackListReport.hashCode() : 0);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null || getClass() != obj.getClass()) {
+        return false;
+      }
+      final BlackListInfo that = (BlackListInfo) obj;
+      if (trackerName == null ? that.trackerName != null :
+          !trackerName.equals(that.trackerName)) {
+        return false;
+      }
+      if (reasonForBlackListing == null ? that.reasonForBlackListing != null :
+          !reasonForBlackListing.equals(that.reasonForBlackListing)) {
+        return false;
+      }
+      if (blackListReport == null ? that.blackListReport != null :
+          !blackListReport.equals(that.blackListReport)) {
+        return false;
+      }
+      return true;
+    }
   }
   
   public static final long UNINITIALIZED_MEMORY_VALUE = -1;

+ 6 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java

@@ -137,7 +137,12 @@ public class NotRunningJob implements MRClientProtocol {
     jobReport.setJobState(jobState);
     jobReport.setUser(applicationReport.getUser());
     jobReport.setStartTime(applicationReport.getStartTime());
-    jobReport.setDiagnostics(applicationReport.getDiagnostics());
+    YarnApplicationState state = applicationReport.getYarnApplicationState();
+    if (state == YarnApplicationState.KILLED
+        || state == YarnApplicationState.FAILED
+        || state == YarnApplicationState.FINISHED) {
+      jobReport.setDiagnostics(applicationReport.getDiagnostics());
+    }
     jobReport.setJobName(applicationReport.getName());
     jobReport.setTrackingUrl(applicationReport.getTrackingUrl());
     jobReport.setFinishTime(applicationReport.getFinishTime());

+ 22 - 8
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);

+ 8 - 9
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();
   }
 
   /**

+ 12 - 3
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(),

+ 4 - 0
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");

+ 44 - 45
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java

@@ -155,9 +155,9 @@ public class TestNetworkedJob {
       // test getters
       assertTrue(runningJob.getConfiguration().toString()
           .endsWith("0001/job.xml"));
-      assertEquals(runningJob.getID(), jobId);
-      assertEquals(runningJob.getJobID(), jobId.toString());
-      assertEquals(runningJob.getJobName(), "N/A");
+      assertEquals(jobId, runningJob.getID());
+      assertEquals(jobId.toString(), runningJob.getJobID());
+      assertEquals("N/A", runningJob.getJobName());
       assertTrue(runningJob.getJobFile().endsWith(
           ".staging/" + runningJob.getJobID() + "/job.xml"));
       assertTrue(runningJob.getTrackingURL().length() > 0);
@@ -169,40 +169,40 @@ public class TestNetworkedJob {
       TaskCompletionEvent[] tce = runningJob.getTaskCompletionEvents(0);
       assertEquals(tce.length, 0);
 
-      assertEquals(runningJob.getHistoryUrl(),"");
+      assertEquals("", runningJob.getHistoryUrl());
       assertFalse(runningJob.isRetired());
-      assertEquals( runningJob.getFailureInfo(),"");
-      assertEquals(runningJob.getJobStatus().getJobName(), "N/A");
-      assertEquals(client.getMapTaskReports(jobId).length, 0);
+      assertEquals("", runningJob.getFailureInfo());
+      assertEquals("N/A", runningJob.getJobStatus().getJobName());
+      assertEquals(0, client.getMapTaskReports(jobId).length);
       
       try {
         client.getSetupTaskReports(jobId);
       } catch (YarnRuntimeException e) {
-        assertEquals(e.getMessage(), "Unrecognized task type: JOB_SETUP");
+        assertEquals("Unrecognized task type: JOB_SETUP", e.getMessage());
       }
       try {
         client.getCleanupTaskReports(jobId);
       } catch (YarnRuntimeException e) {
-        assertEquals(e.getMessage(), "Unrecognized task type: JOB_CLEANUP");
+        assertEquals("Unrecognized task type: JOB_CLEANUP", e.getMessage());
       }
-      assertEquals(client.getReduceTaskReports(jobId).length, 0);
+      assertEquals(0, client.getReduceTaskReports(jobId).length);
       // test ClusterStatus
       ClusterStatus status = client.getClusterStatus(true);
-      assertEquals(status.getActiveTrackerNames().size(), 2);
+      assertEquals(2, status.getActiveTrackerNames().size());
       // it method does not implemented and always return empty array or null;
-      assertEquals(status.getBlacklistedTrackers(), 0);
-      assertEquals(status.getBlacklistedTrackerNames().size(), 0);
-      assertEquals(status.getBlackListedTrackersInfo().size(), 0);
-      assertEquals(status.getJobTrackerStatus(), JobTrackerStatus.RUNNING);
-      assertEquals(status.getMapTasks(), 1);
-      assertEquals(status.getMaxMapTasks(), 20);
-      assertEquals(status.getMaxReduceTasks(), 4);
-      assertEquals(status.getNumExcludedNodes(), 0);
-      assertEquals(status.getReduceTasks(), 1);
-      assertEquals(status.getTaskTrackers(), 2);
-      assertEquals(status.getTTExpiryInterval(), 0);
-      assertEquals(status.getJobTrackerStatus(), JobTrackerStatus.RUNNING);
-      assertEquals(status.getGraylistedTrackers(), 0);
+      assertEquals(0, status.getBlacklistedTrackers());
+      assertEquals(0, status.getBlacklistedTrackerNames().size());
+      assertEquals(0, status.getBlackListedTrackersInfo().size());
+      assertEquals(JobTrackerStatus.RUNNING, status.getJobTrackerStatus());
+      assertEquals(1, status.getMapTasks());
+      assertEquals(20, status.getMaxMapTasks());
+      assertEquals(4, status.getMaxReduceTasks());
+      assertEquals(0, status.getNumExcludedNodes());
+      assertEquals(1, status.getReduceTasks());
+      assertEquals(2, status.getTaskTrackers());
+      assertEquals(0, status.getTTExpiryInterval());
+      assertEquals(JobTrackerStatus.RUNNING, status.getJobTrackerStatus());
+      assertEquals(0, status.getGraylistedTrackers());
 
       // test read and write
       ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
@@ -219,34 +219,34 @@ public class TestNetworkedJob {
 
       // test taskStatusfilter
       JobClient.setTaskOutputFilter(job, TaskStatusFilter.ALL);
-      assertEquals(JobClient.getTaskOutputFilter(job), TaskStatusFilter.ALL);
+      assertEquals(TaskStatusFilter.ALL, JobClient.getTaskOutputFilter(job));
 
       // runningJob.setJobPriority(JobPriority.HIGH.name());
 
       // test default map
-      assertEquals(client.getDefaultMaps(), 20);
-      assertEquals(client.getDefaultReduces(), 4);
-      assertEquals(client.getSystemDir().getName(), "jobSubmitDir");
+      assertEquals(20, client.getDefaultMaps());
+      assertEquals(4, client.getDefaultReduces());
+      assertEquals("jobSubmitDir", client.getSystemDir().getName());
       // test queue information
       JobQueueInfo[] rootQueueInfo = client.getRootQueues();
-      assertEquals(rootQueueInfo.length, 1);
-      assertEquals(rootQueueInfo[0].getQueueName(), "default");
+      assertEquals(1, rootQueueInfo.length);
+      assertEquals("default", rootQueueInfo[0].getQueueName());
       JobQueueInfo[] qinfo = client.getQueues();
-      assertEquals(qinfo.length, 1);
-      assertEquals(qinfo[0].getQueueName(), "default");
-      assertEquals(client.getChildQueues("default").length, 0);
-      assertEquals(client.getJobsFromQueue("default").length, 1);
+      assertEquals(1, qinfo.length);
+      assertEquals("default", qinfo[0].getQueueName());
+      assertEquals(0, client.getChildQueues("default").length);
+      assertEquals(1, client.getJobsFromQueue("default").length);
       assertTrue(client.getJobsFromQueue("default")[0].getJobFile().endsWith(
           "/job.xml"));
 
       JobQueueInfo qi = client.getQueueInfo("default");
-      assertEquals(qi.getQueueName(), "default");
-      assertEquals(qi.getQueueState(), "running");
+      assertEquals("default", qi.getQueueName());
+      assertEquals("running", qi.getQueueState());
 
       QueueAclsInfo[] aai = client.getQueueAclsForCurrentUser();
-      assertEquals(aai.length, 2);
-      assertEquals(aai[0].getQueueName(), "root");
-      assertEquals(aai[1].getQueueName(), "default");
+      assertEquals(2, aai.length);
+      assertEquals("root", aai[0].getQueueName());
+      assertEquals("default", aai[1].getQueueName());
       
       // test JobClient
       // The following asserts read JobStatus twice and ensure the returned
@@ -282,12 +282,11 @@ public class TestNetworkedJob {
     BlackListInfo info2 = new BlackListInfo();
     info2.readFields(new DataInputStream(new ByteArrayInputStream(byteOut
         .toByteArray())));
-    assertEquals(info, info);
-    assertEquals(info.toString(), info.toString());
-    assertEquals(info.getTrackerName(), "trackerName");
-    assertEquals(info.getReasonForBlackListing(), "reasonForBlackListing");
-    assertEquals(info.getBlackListReport(), "blackListInfo");
-
+    assertEquals(info, info2);
+    assertEquals(info.toString(), info2.toString());
+    assertEquals("trackerName", info2.getTrackerName());
+    assertEquals("reasonForBlackListing", info2.getReasonForBlackListing());
+    assertEquals("blackListInfo", info2.getBlackListReport());
   }
 /**
  *  test run from command line JobQueueClient

+ 22 - 9
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();

+ 3 - 0
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());

+ 19 - 1
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");

+ 2 - 0
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;

+ 10 - 7
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,

+ 25 - 1
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);
 

+ 20 - 1
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);
   }

+ 9 - 1
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);

+ 5 - 1
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();

+ 2 - 0
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 

+ 9 - 0
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 = "";

+ 6 - 0
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");

+ 4 - 0
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");

+ 29 - 11
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";

+ 4 - 2
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,

+ 7 - 1
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");
 

+ 11 - 1
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);
   }

+ 12 - 4
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, 

+ 16 - 1
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);

+ 14 - 6
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);

+ 12 - 3
hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/TestWordStats.java

@@ -31,15 +31,18 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestWordStats {
 
   private final static String INPUT = "src/test/java/org/apache/hadoop/examples/pi/math";
-  private final static String MEAN_OUTPUT = "build/data/mean_output";
-  private final static String MEDIAN_OUTPUT = "build/data/median_output";
-  private final static String STDDEV_OUTPUT = "build/data/stddev_output";
+  private final static String BASEDIR = System.getProperty("test.build.data",
+                                                           "target/test-dir");
+  private final static String MEAN_OUTPUT = BASEDIR + "/mean_output";
+  private final static String MEDIAN_OUTPUT = BASEDIR + "/median_output";
+  private final static String STDDEV_OUTPUT = BASEDIR + "/stddev_output";
 
   /**
    * Modified internal test class that is designed to read all the files in the
@@ -286,4 +289,10 @@ public class TestWordStats {
     assertEquals(stddev, wr.read(INPUT), 0.0);
   }
 
+  @AfterClass public static void cleanup() throws Exception {
+    deleteDir(new File(MEAN_OUTPUT));
+    deleteDir(new File(MEDIAN_OUTPUT));
+    deleteDir(new File(STDDEV_OUTPUT));
+  }
+
 }

+ 10 - 1
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);

+ 13 - 0
hadoop-project/pom.xml

@@ -296,6 +296,19 @@
         <version>${project.version}</version>
       </dependency>
 
+      <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>

+ 21 - 0
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>

+ 45 - 23
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) {

+ 79 - 0
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;
+  }
+}

+ 19 - 0
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;

+ 106 - 14
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",

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java

@@ -217,7 +217,7 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
   protected void verifyConnections() throws InterruptedException,
       YarnException {
     assertTrue("NMs failed to connect to the RM",
-        cluster.waitForNodeManagersToConnect(20000));
+        cluster.waitForNodeManagersToConnect(5000));
     verifyClientConnection();
   }
 
@@ -279,7 +279,6 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     cluster.resetStartFailoverFlag(false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMFailover.java

@@ -142,7 +142,6 @@ public class TestRMFailover extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 
@@ -231,7 +230,6 @@ public class TestRMFailover extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster.init(conf);
     cluster.start();
-    getAdminService(0).transitionToActive(req);
     assertFalse("RM never turned active", -1 == cluster.getActiveRMIndex());
     verifyConnections();
 

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java

@@ -1195,6 +1195,24 @@ public class TestYarnClient {
       client.init(yarnConf);
       client.start();
 
+      int attempts;
+      for(attempts = 10; attempts > 0; attempts--) {
+        if (cluster.getResourceManager().getRMContext().getReservationSystem()
+            .getPlan(ReservationSystemTestUtil.reservationQ).getTotalCapacity()
+            .getMemory() > 0) {
+          break;
+        }
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+      }
+      if (attempts <= 0) {
+        Assert.fail("Exhausted attempts in checking if node capacity was "
+            + "added to the plan");
+      }
+
       // create a reservation
       Clock clock = new UTCClock();
       long arrival = clock.getTime();

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/FileSystemTimelineWriter.java

@@ -305,8 +305,10 @@ public class FileSystemTimelineWriter extends TimelineWriter{
     Path domainLogPath =
         new Path(createAttemptDir(appAttemptId), DOMAIN_LOG_PREFIX
             + appAttemptId.toString());
-    LOG.info("Writing domains for " + appAttemptId.toString() + " to "
-        + domainLogPath);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Writing domains for " + appAttemptId.toString() + " to "
+          + domainLogPath);
+    }
     this.logFDsCache.writeDomainLog(
         fs, domainLogPath, objMapper, domain, isAppendSupported);
   }

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác