瀏覽代碼

Merging trunk to HDFS-2802 branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1403301 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 年之前
父節點
當前提交
564adec5b5
共有 73 個文件被更改,包括 2928 次插入1154 次删除
  1. 6 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto
  3. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/IpcConnectionContext.proto
  4. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto
  5. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto
  6. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/ZKFCProtocol.proto
  7. 1 0
      hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto
  8. 6 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
  9. 1 0
      hadoop-common-project/hadoop-common/src/test/proto/test.proto
  10. 1 0
      hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
  11. 18 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  12. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
  13. 318 181
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.c
  14. 230 44
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.h
  15. 333 185
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.c
  16. 215 16
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.h
  17. 478 267
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c
  18. 141 21
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.h
  19. 379 230
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  21. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  24. 27 33
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  25. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  26. 12 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  27. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  28. 8 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  29. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
  30. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  32. 12 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  33. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  34. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  35. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  36. 9 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  37. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java
  38. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  39. 5 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  42. 26 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java
  44. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  45. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java
  46. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c
  47. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto
  48. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  49. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  50. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto
  51. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HAZKInfo.proto
  52. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
  53. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
  54. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
  55. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
  56. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto
  57. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto
  58. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto
  59. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  60. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
  61. 2 0
      hadoop-mapreduce-project/CHANGES.txt
  62. 4 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  63. 34 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
  64. 2 0
      hadoop-yarn-project/CHANGES.txt
  65. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  66. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java
  67. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  68. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  69. 136 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java
  70. 200 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
  71. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
  72. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerInfo.java
  73. 153 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java

+ 6 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -330,6 +330,9 @@ Release 2.0.3-alpha - Unreleased
 
 
     HADOOP-8925. Remove the packaging. (eli)
     HADOOP-8925. Remove the packaging. (eli)
 
 
+    HADOOP-8985. Add namespace declarations in .proto files for languages 
+    other than java. (Binglin Chan via suresh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang
     HADOOP-8866. SampleQuantiles#query is O(N^2) instead of O(N). (Andrew Wang
@@ -383,6 +386,9 @@ Release 2.0.3-alpha - Unreleased
     HADOOP-8951. RunJar to fail with user-comprehensible error 
     HADOOP-8951. RunJar to fail with user-comprehensible error 
     message if jar missing. (stevel via suresh)
     message if jar missing. (stevel via suresh)
 
 
+    HADOOP-8713. TestRPCCompatibility fails intermittently with JDK7
+    (Trevor Robinson via tgraves)
+
 Release 2.0.2-alpha - 2012-09-07 
 Release 2.0.2-alpha - 2012-09-07 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/HAServiceProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.ha.proto";
 option java_outer_classname = "HAServiceProtocolProtos";
 option java_outer_classname = "HAServiceProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 enum HAServiceStateProto {
 enum HAServiceStateProto {
   INITIALIZING = 0;
   INITIALIZING = 0;

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/IpcConnectionContext.proto

@@ -18,6 +18,7 @@
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "IpcConnectionContextProtos";
 option java_outer_classname = "IpcConnectionContextProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 /**
 /**
  * Spec for UserInformationProto is specified in ProtoUtil#makeIpcConnectionContext
  * Spec for UserInformationProto is specified in ProtoUtil#makeIpcConnectionContext

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/ProtocolInfo.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "ProtocolInfoProtos";
 option java_outer_classname = "ProtocolInfoProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 /**
 /**
  * Request to get protocol versions for all supported rpc kinds.
  * Request to get protocol versions for all supported rpc kinds.

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto

@@ -18,6 +18,7 @@
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "RpcPayloadHeaderProtos";
 option java_outer_classname = "RpcPayloadHeaderProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 /**
 /**
  * This is the rpc payload header. It is sent with every rpc call.
  * This is the rpc payload header. It is sent with every rpc call.

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/ZKFCProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.ha.proto";
 option java_outer_classname = "ZKFCProtocolProtos";
 option java_outer_classname = "ZKFCProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 message CedeActiveRequestProto {
 message CedeActiveRequestProto {
   required uint32 millisToCede = 1;
   required uint32 millisToCede = 1;

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/proto/hadoop_rpc.proto

@@ -23,6 +23,7 @@
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "HadoopRpcProtos";
 option java_outer_classname = "HadoopRpcProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 /**
 /**
  * This message is used for Protobuf Rpc Engine.
  * This message is used for Protobuf Rpc Engine.

+ 6 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRes
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /** Unit test for supporting method-name based compatible RPCs. */
 /** Unit test for supporting method-name based compatible RPCs. */
@@ -114,6 +115,11 @@ public class TestRPCCompatibility {
     }
     }
 
 
   }
   }
+
+  @Before
+  public void setUp() {
+    ProtocolSignature.resetCache();
+  }
   
   
   @After
   @After
   public void tearDown() throws IOException {
   public void tearDown() throws IOException {
@@ -219,7 +225,6 @@ System.out.println("echo int is NOT supported");
   
   
   @Test // equal version client and server
   @Test // equal version client and server
   public void testVersion2ClientVersion2Server() throws Exception {
   public void testVersion2ClientVersion2Server() throws Exception {
-    ProtocolSignature.resetCache();
     // create a server with two handlers
     // create a server with two handlers
     TestImpl2 impl = new TestImpl2();
     TestImpl2 impl = new TestImpl2();
     server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
     server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/proto/test.proto

@@ -19,6 +19,7 @@
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "TestProtos";
 option java_outer_classname = "TestProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 message EmptyRequestProto {
 message EmptyRequestProto {
 }
 }

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto

@@ -19,6 +19,7 @@ option java_package = "org.apache.hadoop.ipc.protobuf";
 option java_outer_classname = "TestRpcServiceProtos";
 option java_outer_classname = "TestRpcServiceProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 
 import "test.proto";
 import "test.proto";
 
 

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -152,6 +152,9 @@ Trunk (Unreleased)
 
 
     HDFS-4110. Refine a log printed in JNStorage. (Liang Xie via suresh)
     HDFS-4110. Refine a log printed in JNStorage. (Liang Xie via suresh)
 
 
+    HDFS-4122. Cleanup HDFS logs and reduce the size of logged messages.
+    (suresh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -238,6 +241,12 @@ Trunk (Unreleased)
     HDFS-4067. TestUnderReplicatedBlocks intermittently fails due to 
     HDFS-4067. TestUnderReplicatedBlocks intermittently fails due to 
     ReplicaAlreadyExistsException. (Jing Zhao via suresh)
     ReplicaAlreadyExistsException. (Jing Zhao via suresh)
 
 
+    HDFS-4115. TestHDFSCLI.testAll fails one test due to number format.
+    (Trevor Robinson via suresh)
+
+    HDFS-4106. BPServiceActor#lastHeartbeat, lastBlockReport and
+    lastDeletedReport should be volatile. (Jing Zhao via suresh)
+
   BREAKDOWN OF HDFS-3077 SUBTASKS
   BREAKDOWN OF HDFS-3077 SUBTASKS
 
 
     HDFS-3077. Quorum-based protocol for reading and writing edit logs.
     HDFS-3077. Quorum-based protocol for reading and writing edit logs.
@@ -425,6 +434,9 @@ Release 2.0.3-alpha - Unreleased
     method for casting; avoiding creation of new empty lists; cleaning up 
     method for casting; avoiding creation of new empty lists; cleaning up 
     some code and rewriting some javadoc. (szetszwo)
     some code and rewriting some javadoc. (szetszwo)
 
 
+    HDFS-4121. Add namespace declarations in hdfs .proto files for languages 
+    other than java. (Binglin Chang via suresh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -504,6 +516,9 @@ Release 2.0.3-alpha - Unreleased
     out stream returned by WebHdfsFileSystem does not support it. (Jing Zhao
     out stream returned by WebHdfsFileSystem does not support it. (Jing Zhao
     via szetszwo)
     via szetszwo)
 
 
+    HDFS-3616. Fix a ConcurrentModificationException bug that BP actor threads
+    may not be shutdown properly in DataNode.  (Jing Zhao via szetszwo)
+
 Release 2.0.2-alpha - 2012-09-07 
 Release 2.0.2-alpha - 2012-09-07 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -746,6 +761,9 @@ Release 2.0.2-alpha - 2012-09-07
     HDFS-3907. Allow multiple users for local block readers. (eli)
     HDFS-3907. Allow multiple users for local block readers. (eli)
 
 
     HDFS-3910. DFSTestUtil#waitReplication should timeout. (eli)
     HDFS-3910. DFSTestUtil#waitReplication should timeout. (eli)
+    
+    HDFS-3920. libwebdhfs string processing and using strerror consistently
+    to handle all errors. (Jing Zhao via suresh)
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto

@@ -22,6 +22,7 @@
 option java_package = "org.apache.hadoop.contrib.bkjournal";
 option java_package = "org.apache.hadoop.contrib.bkjournal";
 option java_outer_classname = "BKJournalProtos";
 option java_outer_classname = "BKJournalProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 318 - 181
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.c

@@ -15,28 +15,43 @@
  * See the License for the specific language governing permissions and
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * limitations under the License.
  */
  */
+
 #include <stdlib.h>
 #include <stdlib.h>
 #include <string.h>
 #include <string.h>
 #include <curl/curl.h>
 #include <curl/curl.h>
-#include <pthread.h>
+
 #include "hdfs_http_client.h"
 #include "hdfs_http_client.h"
 
 
 static pthread_mutex_t curlInitMutex = PTHREAD_MUTEX_INITIALIZER;
 static pthread_mutex_t curlInitMutex = PTHREAD_MUTEX_INITIALIZER;
 static volatile int curlGlobalInited = 0;
 static volatile int curlGlobalInited = 0;
 
 
-ResponseBuffer initResponseBuffer() {
-    ResponseBuffer info = (ResponseBuffer) calloc(1, sizeof(ResponseBufferInternal));
+const char *hdfs_strerror(int errnoval)
+{
+    const char *msg = NULL;
+    if (errnoval < 0 || errnoval >= sys_nerr) {
+        msg = "Invalid Error Code";
+    } else if (sys_errlist == NULL) {
+        msg = "Unknown Error";
+    } else {
+        msg = sys_errlist[errnoval];
+    }
+    return msg;
+}
+
+int initResponseBuffer(struct ResponseBuffer **buffer)
+{
+    struct ResponseBuffer *info = NULL;
+    int ret = 0;
+    info = calloc(1, sizeof(struct ResponseBuffer));
     if (!info) {
     if (!info) {
-        fprintf(stderr, "Cannot allocate memory for responseInfo\n");
-        return NULL;
+        ret = ENOMEM;
     }
     }
-    info->remaining = 0;
-    info->offset = 0;
-    info->content = NULL;
-    return info;
+    *buffer = info;
+    return ret;
 }
 }
 
 
-void freeResponseBuffer(ResponseBuffer buffer) {
+void freeResponseBuffer(struct ResponseBuffer *buffer)
+{
     if (buffer) {
     if (buffer) {
         if (buffer->content) {
         if (buffer->content) {
             free(buffer->content);
             free(buffer->content);
@@ -46,8 +61,9 @@ void freeResponseBuffer(ResponseBuffer buffer) {
     }
     }
 }
 }
 
 
-void freeResponse(Response resp)  {
-    if(resp) {
+void freeResponse(struct Response *resp)
+{
+    if (resp) {
         freeResponseBuffer(resp->body);
         freeResponseBuffer(resp->body);
         freeResponseBuffer(resp->header);
         freeResponseBuffer(resp->header);
         free(resp);
         free(resp);
@@ -55,21 +71,30 @@ void freeResponse(Response resp)  {
     }
     }
 }
 }
 
 
-/* Callback for allocating local buffer and reading data to local buffer */
-static size_t writefunc(void *ptr, size_t size, size_t nmemb, ResponseBuffer rbuffer) {
+/** 
+ * Callback used by libcurl for allocating local buffer and 
+ * reading data to local buffer
+ */
+static size_t writefunc(void *ptr, size_t size,
+                        size_t nmemb, struct ResponseBuffer *rbuffer)
+{
+    void *temp = NULL;
     if (size * nmemb < 1) {
     if (size * nmemb < 1) {
         return 0;
         return 0;
     }
     }
     if (!rbuffer) {
     if (!rbuffer) {
-        fprintf(stderr, "In writefunc, ResponseBuffer is NULL.\n");
-        return -1;
+        fprintf(stderr,
+                "ERROR: ResponseBuffer is NULL for the callback writefunc.\n");
+        return 0;
     }
     }
     
     
     if (rbuffer->remaining < size * nmemb) {
     if (rbuffer->remaining < size * nmemb) {
-        rbuffer->content = realloc(rbuffer->content, rbuffer->offset + size * nmemb + 1);
-        if (rbuffer->content == NULL) {
-            return -1;
+        temp = realloc(rbuffer->content, rbuffer->offset + size * nmemb + 1);
+        if (temp == NULL) {
+            fprintf(stderr, "ERROR: fail to realloc in callback writefunc.\n");
+            return 0;
         }
         }
+        rbuffer->content = temp;
         rbuffer->remaining = size * nmemb;
         rbuffer->remaining = size * nmemb;
     }
     }
     memcpy(rbuffer->content + rbuffer->offset, ptr, size * nmemb);
     memcpy(rbuffer->content + rbuffer->offset, ptr, size * nmemb);
@@ -80,67 +105,84 @@ static size_t writefunc(void *ptr, size_t size, size_t nmemb, ResponseBuffer rbu
 }
 }
 
 
 /**
 /**
- * Callback for reading data to buffer provided by user, 
+ * Callback used by libcurl for reading data into buffer provided by user,
  * thus no need to reallocate buffer.
  * thus no need to reallocate buffer.
  */
  */
-static size_t writefunc_withbuffer(void *ptr, size_t size, size_t nmemb, ResponseBuffer rbuffer) {
+static size_t writeFuncWithUserBuffer(void *ptr, size_t size,
+                                   size_t nmemb, struct ResponseBuffer *rbuffer)
+{
+    size_t toCopy = 0;
     if (size * nmemb < 1) {
     if (size * nmemb < 1) {
         return 0;
         return 0;
     }
     }
     if (!rbuffer || !rbuffer->content) {
     if (!rbuffer || !rbuffer->content) {
-        fprintf(stderr, "In writefunc_withbuffer, the buffer provided by user is NULL.\n");
+        fprintf(stderr,
+                "ERROR: buffer to read is NULL for the "
+                "callback writeFuncWithUserBuffer.\n");
         return 0;
         return 0;
     }
     }
     
     
-    size_t toCopy = rbuffer->remaining < (size * nmemb) ? rbuffer->remaining : (size * nmemb);
+    toCopy = rbuffer->remaining < (size * nmemb) ?
+                            rbuffer->remaining : (size * nmemb);
     memcpy(rbuffer->content + rbuffer->offset, ptr, toCopy);
     memcpy(rbuffer->content + rbuffer->offset, ptr, toCopy);
     rbuffer->offset += toCopy;
     rbuffer->offset += toCopy;
     rbuffer->remaining -= toCopy;
     rbuffer->remaining -= toCopy;
     return toCopy;
     return toCopy;
 }
 }
 
 
-//callback for writing data to remote peer
-static size_t readfunc(void *ptr, size_t size, size_t nmemb, void *stream) {
+/**
+ * Callback used by libcurl for writing data to remote peer
+ */
+static size_t readfunc(void *ptr, size_t size, size_t nmemb, void *stream)
+{
+    struct webhdfsBuffer *wbuffer = NULL;
     if (size * nmemb < 1) {
     if (size * nmemb < 1) {
-        fprintf(stderr, "In readfunc callback: size * nmemb == %ld\n", size * nmemb);
         return 0;
         return 0;
     }
     }
-    webhdfsBuffer *wbuffer = (webhdfsBuffer *) stream;
     
     
+    wbuffer = stream;
     pthread_mutex_lock(&wbuffer->writeMutex);
     pthread_mutex_lock(&wbuffer->writeMutex);
     while (wbuffer->remaining == 0) {
     while (wbuffer->remaining == 0) {
         /*
         /*
-         * the current remainning bytes to write is 0,
-         * check whether need to finish the transfer
+         * The current remainning bytes to write is 0,
+         * check closeFlag to see whether need to finish the transfer.
          * if yes, return 0; else, wait
          * if yes, return 0; else, wait
          */
          */
-        if (wbuffer->closeFlag) {
-            //we can close the transfer now
+        if (wbuffer->closeFlag) { // We can close the transfer now
+            //For debug
             fprintf(stderr, "CloseFlag is set, ready to close the transfer\n");
             fprintf(stderr, "CloseFlag is set, ready to close the transfer\n");
             pthread_mutex_unlock(&wbuffer->writeMutex);
             pthread_mutex_unlock(&wbuffer->writeMutex);
             return 0;
             return 0;
         } else {
         } else {
-            // len == 0 indicates that user's buffer has been transferred
+            // remaining == 0 but closeFlag is not set
+            // indicates that user's buffer has been transferred
             pthread_cond_signal(&wbuffer->transfer_finish);
             pthread_cond_signal(&wbuffer->transfer_finish);
-            pthread_cond_wait(&wbuffer->newwrite_or_close, &wbuffer->writeMutex);
+            pthread_cond_wait(&wbuffer->newwrite_or_close,
+                                    &wbuffer->writeMutex);
         }
         }
     }
     }
     
     
-    if(wbuffer->remaining > 0 && !wbuffer->closeFlag) {
-        size_t copySize = wbuffer->remaining < size * nmemb ? wbuffer->remaining : size * nmemb;
+    if (wbuffer->remaining > 0 && !wbuffer->closeFlag) {
+        size_t copySize = wbuffer->remaining < size * nmemb ?
+                                wbuffer->remaining : size * nmemb;
         memcpy(ptr, wbuffer->wbuffer + wbuffer->offset, copySize);
         memcpy(ptr, wbuffer->wbuffer + wbuffer->offset, copySize);
         wbuffer->offset += copySize;
         wbuffer->offset += copySize;
         wbuffer->remaining -= copySize;
         wbuffer->remaining -= copySize;
         pthread_mutex_unlock(&wbuffer->writeMutex);
         pthread_mutex_unlock(&wbuffer->writeMutex);
         return copySize;
         return copySize;
     } else {
     } else {
-        fprintf(stderr, "Webhdfs buffer is %ld, it should be a positive value!\n", wbuffer->remaining);
+        fprintf(stderr, "ERROR: webhdfsBuffer's remaining is %ld, "
+                "it should be a positive value!\n", wbuffer->remaining);
         pthread_mutex_unlock(&wbuffer->writeMutex);
         pthread_mutex_unlock(&wbuffer->writeMutex);
         return 0;
         return 0;
     }
     }
 }
 }
 
 
-static void initCurlGlobal() {
+/**
+ * Initialize the global libcurl environment
+ */
+static void initCurlGlobal()
+{
     if (!curlGlobalInited) {
     if (!curlGlobalInited) {
         pthread_mutex_lock(&curlInitMutex);
         pthread_mutex_lock(&curlInitMutex);
         if (!curlGlobalInited) {
         if (!curlGlobalInited) {
@@ -151,202 +193,297 @@ static void initCurlGlobal() {
     }
     }
 }
 }
 
 
-static Response launchCmd(char *url, enum HttpHeader method, enum Redirect followloc) {
-    CURL *curl;
-    CURLcode res;
-    Response resp;
+/**
+ * Launch simple commands (commands without file I/O) and return response
+ *
+ * @param url       Target URL
+ * @param method    HTTP method (GET/PUT/POST)
+ * @param followloc Whether or not need to set CURLOPT_FOLLOWLOCATION
+ * @param response  Response from remote service
+ * @return 0 for success and non-zero value to indicate error
+ */
+static int launchCmd(const char *url, enum HttpHeader method,
+                     enum Redirect followloc, struct Response **response)
+{
+    CURL *curl = NULL;
+    CURLcode curlCode;
+    int ret = 0;
+    struct Response *resp = NULL;
     
     
-    resp = (Response) calloc(1, sizeof(*resp));
+    resp = calloc(1, sizeof(struct Response));
     if (!resp) {
     if (!resp) {
-        return NULL;
+        return ENOMEM;
+    }
+    ret = initResponseBuffer(&(resp->body));
+    if (ret) {
+        goto done;
+    }
+    ret = initResponseBuffer(&(resp->header));
+    if (ret) {
+        goto done;
     }
     }
-    resp->body = initResponseBuffer();
-    resp->header = initResponseBuffer();
     initCurlGlobal();
     initCurlGlobal();
-    curl = curl_easy_init();                     /* get a curl handle */
-    if(curl) {
-        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
-        curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
-        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
-        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
-        curl_easy_setopt(curl, CURLOPT_URL, url);       /* specify target URL */
-        switch(method) {
-            case GET:
-                break;
-            case PUT:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"PUT");
-                break;
-            case POST:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"POST");
-                break;
-            case DELETE:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"DELETE");
-                break;
-            default:
-                fprintf(stderr, "\nHTTP method not defined\n");
-                exit(EXIT_FAILURE);
-        }
-        if(followloc == YES) {
-            curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
-        }
-        
-        res = curl_easy_perform(curl);                 /* Now run the curl handler */
-        if(res != CURLE_OK) {
-            fprintf(stderr, "preform the URL %s failed\n", url);
-            return NULL;
-        }
+    curl = curl_easy_init();
+    if (!curl) {
+        ret = ENOMEM;       // curl_easy_init does not return error code,
+                            // and most of its errors are caused by malloc()
+        fprintf(stderr, "ERROR in curl_easy_init.\n");
+        goto done;
+    }
+    /* Set callback function for reading data from remote service */
+    curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
+    curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
+    curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+    curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
+    curl_easy_setopt(curl, CURLOPT_URL, url);
+    switch(method) {
+        case GET:
+            break;
+        case PUT:
+            curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, "PUT");
+            break;
+        case POST:
+            curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, "POST");
+            break;
+        case DELETE:
+            curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, "DELETE");
+            break;
+        default:
+            ret = EINVAL;
+            fprintf(stderr, "ERROR: Invalid HTTP method\n");
+            goto done;
+    }
+    if (followloc == YES) {
+        curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
+    }
+    /* Now run the curl handler */
+    curlCode = curl_easy_perform(curl);
+    if (curlCode != CURLE_OK) {
+        ret = EIO;
+        fprintf(stderr, "ERROR: preform the URL %s failed, <%d>: %s\n",
+                url, curlCode, curl_easy_strerror(curlCode));
+    }
+done:
+    if (curl != NULL) {
         curl_easy_cleanup(curl);
         curl_easy_cleanup(curl);
     }
     }
-    return resp;
+    if (ret) {
+        free(resp);
+        resp = NULL;
+    }
+    *response = resp;
+    return ret;
 }
 }
 
 
-static Response launchRead_internal(char *url, enum HttpHeader method, enum Redirect followloc, Response resp) {
+/**
+ * Launch the read request. The request is sent to the NameNode and then 
+ * redirected to corresponding DataNode
+ *
+ * @param url   The URL for the read request
+ * @param resp  The response containing the buffer provided by user
+ * @return 0 for success and non-zero value to indicate error
+ */
+static int launchReadInternal(const char *url, struct Response* resp)
+{
+    CURL *curl;
+    CURLcode curlCode;
+    int ret = 0;
+    
     if (!resp || !resp->body || !resp->body->content) {
     if (!resp || !resp->body || !resp->body->content) {
-        fprintf(stderr, "The user provided buffer should not be NULL!\n");
-        return NULL;
+        fprintf(stderr,
+                "ERROR: invalid user-provided buffer!\n");
+        return EINVAL;
     }
     }
     
     
-    CURL *curl;
-    CURLcode res;
     initCurlGlobal();
     initCurlGlobal();
-    curl = curl_easy_init();                     /* get a curl handle */
-    if(curl) {
-        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc_withbuffer);
-        curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
-        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
-        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
-        curl_easy_setopt(curl, CURLOPT_URL, url);       /* specify target URL */
-        if(followloc == YES) {
-            curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
-        }
-        
-        res = curl_easy_perform(curl);                 /* Now run the curl handler */
-        if(res != CURLE_OK && res != CURLE_PARTIAL_FILE) {
-            fprintf(stderr, "preform the URL %s failed\n", url);
-            return NULL;
-        }
-        curl_easy_cleanup(curl);
+    /* get a curl handle */
+    curl = curl_easy_init();
+    if (!curl) {
+        fprintf(stderr, "ERROR in curl_easy_init.\n");
+        return ENOMEM;
     }
     }
-    return resp;
-
+    curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writeFuncWithUserBuffer);
+    curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
+    curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+    curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
+    curl_easy_setopt(curl, CURLOPT_URL, url);
+    curl_easy_setopt(curl, CURLOPT_FOLLOWLOCATION, 1);
+    
+    curlCode = curl_easy_perform(curl);
+    if (curlCode != CURLE_OK && curlCode != CURLE_PARTIAL_FILE) {
+        ret = EIO;
+        fprintf(stderr, "ERROR: preform the URL %s failed, <%d>: %s\n",
+                url, curlCode, curl_easy_strerror(curlCode));
+    }
+    
+    curl_easy_cleanup(curl);
+    return ret;
 }
 }
 
 
-static Response launchWrite(const char *url, enum HttpHeader method, webhdfsBuffer *uploadBuffer) {
+/**
+ * The function does the write operation by connecting to a DataNode. 
+ * The function keeps the connection with the DataNode until 
+ * the closeFlag is set. Whenever the current data has been sent out, 
+ * the function blocks waiting for further input from user or close.
+ *
+ * @param url           URL of the remote DataNode
+ * @param method        PUT for create and POST for append
+ * @param uploadBuffer  Buffer storing user's data to write
+ * @param response      Response from remote service
+ * @return 0 for success and non-zero value to indicate error
+ */
+static int launchWrite(const char *url, enum HttpHeader method,
+                       struct webhdfsBuffer *uploadBuffer,
+                       struct Response **response)
+{
+    CURLcode curlCode;
+    struct Response* resp = NULL;
+    struct curl_slist *chunk = NULL;
+    CURL *curl = NULL;
+    int ret = 0;
+    
     if (!uploadBuffer) {
     if (!uploadBuffer) {
-        fprintf(stderr, "upload buffer is NULL!\n");
-        errno = EINVAL;
-        return NULL;
+        fprintf(stderr, "ERROR: upload buffer is NULL!\n");
+        return EINVAL;
     }
     }
+    
     initCurlGlobal();
     initCurlGlobal();
-    CURLcode res;
-    Response response = (Response) calloc(1, sizeof(*response));
-    if (!response) {
-        fprintf(stderr, "failed to allocate memory for response\n");
-        return NULL;
-    }
-    response->body = initResponseBuffer();
-    response->header = initResponseBuffer();
+    resp = calloc(1, sizeof(struct Response));
+    if (!resp) {
+        return ENOMEM;
+    }
+    ret = initResponseBuffer(&(resp->body));
+    if (ret) {
+        goto done;
+    }
+    ret = initResponseBuffer(&(resp->header));
+    if (ret) {
+        goto done;
+    }
     
     
-    //connect to the datanode in order to create the lease in the namenode
-    CURL *curl = curl_easy_init();
+    // Connect to the datanode in order to create the lease in the namenode
+    curl = curl_easy_init();
     if (!curl) {
     if (!curl) {
-        fprintf(stderr, "Failed to initialize the curl handle.\n");
-        return NULL;
+        fprintf(stderr, "ERROR: failed to initialize the curl handle.\n");
+        return ENOMEM;
     }
     }
     curl_easy_setopt(curl, CURLOPT_URL, url);
     curl_easy_setopt(curl, CURLOPT_URL, url);
     
     
-    if(curl) {
-        curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
-        curl_easy_setopt(curl, CURLOPT_WRITEDATA, response->body);
-        curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
-        curl_easy_setopt(curl, CURLOPT_WRITEHEADER, response->header);
-        curl_easy_setopt(curl, CURLOPT_READFUNCTION, readfunc);
-        curl_easy_setopt(curl, CURLOPT_READDATA, uploadBuffer);
-        curl_easy_setopt(curl, CURLOPT_UPLOAD, 1L);
-        curl_easy_setopt(curl, CURLOPT_VERBOSE, 1);
-        
-        struct curl_slist *chunk = NULL;
-        chunk = curl_slist_append(chunk, "Transfer-Encoding: chunked");
-        res = curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
-        chunk = curl_slist_append(chunk, "Expect:");
-        res = curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
-        
-        switch(method) {
-            case GET:
-                break;
-            case PUT:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"PUT");
-                break;
-            case POST:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"POST");
-                break;
-            case DELETE:
-                curl_easy_setopt(curl,CURLOPT_CUSTOMREQUEST,"DELETE");
-                break;
-            default:
-                fprintf(stderr, "\nHTTP method not defined\n");
-                exit(EXIT_FAILURE);
-        }
-        res = curl_easy_perform(curl);
+    curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, writefunc);
+    curl_easy_setopt(curl, CURLOPT_WRITEDATA, resp->body);
+    curl_easy_setopt(curl, CURLOPT_HEADERFUNCTION, writefunc);
+    curl_easy_setopt(curl, CURLOPT_WRITEHEADER, resp->header);
+    curl_easy_setopt(curl, CURLOPT_READFUNCTION, readfunc);
+    curl_easy_setopt(curl, CURLOPT_READDATA, uploadBuffer);
+    curl_easy_setopt(curl, CURLOPT_UPLOAD, 1L);
+    
+    chunk = curl_slist_append(chunk, "Transfer-Encoding: chunked");
+    curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
+    chunk = curl_slist_append(chunk, "Expect:");
+    curl_easy_setopt(curl, CURLOPT_HTTPHEADER, chunk);
+    
+    switch(method) {
+        case PUT:
+            curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, "PUT");
+            break;
+        case POST:
+            curl_easy_setopt(curl, CURLOPT_CUSTOMREQUEST, "POST");
+            break;
+        default:
+            ret = EINVAL;
+            fprintf(stderr, "ERROR: Invalid HTTP method\n");
+            goto done;
+    }
+    curlCode = curl_easy_perform(curl);
+    if (curlCode != CURLE_OK) {
+        ret = EIO;
+        fprintf(stderr, "ERROR: preform the URL %s failed, <%d>: %s\n",
+                url, curlCode, curl_easy_strerror(curlCode));
+    }
+    
+done:
+    if (chunk != NULL) {
         curl_slist_free_all(chunk);
         curl_slist_free_all(chunk);
+    }
+    if (curl != NULL) {
         curl_easy_cleanup(curl);
         curl_easy_cleanup(curl);
     }
     }
-    
-    return response;
+    if (ret) {
+        free(resp);
+        resp = NULL;
+    }
+    *response = resp;
+    return ret;
 }
 }
 
 
-Response launchMKDIR(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchMKDIR(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchRENAME(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchRENAME(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchGFS(char *url) {
-    return launchCmd(url, GET, NO);
+int launchGFS(const char *url, struct Response **resp)
+{
+    return launchCmd(url, GET, NO, resp);
 }
 }
 
 
-Response launchLS(char *url) {
-    return launchCmd(url, GET, NO);
+int launchLS(const char *url, struct Response **resp)
+{
+    return launchCmd(url, GET, NO, resp);
 }
 }
 
 
-Response launchCHMOD(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchCHMOD(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchCHOWN(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchCHOWN(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchDELETE(char *url) {
-    return launchCmd(url, DELETE, NO);
+int launchDELETE(const char *url, struct Response **resp)
+{
+    return launchCmd(url, DELETE, NO, resp);
 }
 }
 
 
-Response launchOPEN(char *url, Response resp) {
-    return launchRead_internal(url, GET, YES, resp);
+int launchOPEN(const char *url, struct Response* resp)
+{
+    return launchReadInternal(url, resp);
 }
 }
 
 
-Response launchUTIMES(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchUTIMES(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchNnWRITE(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchNnWRITE(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }
 
 
-Response launchNnAPPEND(char *url) {
-    return launchCmd(url, POST, NO);
+int launchNnAPPEND(const char *url, struct Response **resp)
+{
+    return launchCmd(url, POST, NO, resp);
 }
 }
 
 
-Response launchDnWRITE(const char *url, webhdfsBuffer *buffer) {
-    return launchWrite(url, PUT, buffer);
+int launchDnWRITE(const char *url, struct webhdfsBuffer *buffer,
+                               struct Response **resp)
+{
+    return launchWrite(url, PUT, buffer, resp);
 }
 }
 
 
-Response launchDnAPPEND(const char *url, webhdfsBuffer *buffer) {
-    return launchWrite(url, POST, buffer);
+int launchDnAPPEND(const char *url, struct webhdfsBuffer *buffer,
+                                struct Response **resp)
+{
+    return launchWrite(url, POST, buffer, resp);
 }
 }
 
 
-Response launchSETREPLICATION(char *url) {
-    return launchCmd(url, PUT, NO);
+int launchSETREPLICATION(const char *url, struct Response **resp)
+{
+    return launchCmd(url, PUT, NO, resp);
 }
 }

+ 230 - 44
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_client.h

@@ -26,6 +26,7 @@
 #include <pthread.h> /* for pthread_t */
 #include <pthread.h> /* for pthread_t */
 #include <unistd.h> /* for size_t */
 #include <unistd.h> /* for size_t */
 
 
+/** enum indicating the type of hdfs stream */
 enum hdfsStreamType
 enum hdfsStreamType
 {
 {
     UNINITIALIZED = 0,
     UNINITIALIZED = 0,
@@ -36,28 +37,39 @@ enum hdfsStreamType
 /**
 /**
  * webhdfsBuffer - used for hold the data for read/write from/to http connection
  * webhdfsBuffer - used for hold the data for read/write from/to http connection
  */
  */
-typedef struct {
-    const char *wbuffer;  // The user's buffer for uploading
-    size_t remaining;     // Length of content
-    size_t offset;        // offset for reading
-    int openFlag;         // Check whether the hdfsOpenFile has been called before
-    int closeFlag;        // Whether to close the http connection for writing
-    pthread_mutex_t writeMutex; // Synchronization between the curl and hdfsWrite threads
-    pthread_cond_t newwrite_or_close; // Transferring thread waits for this condition
-                                      // when there is no more content for transferring in the buffer
-    pthread_cond_t transfer_finish; // Condition used to indicate finishing transferring (one buffer)
-} webhdfsBuffer;
+struct webhdfsBuffer {
+    const char *wbuffer;  /* The user's buffer for uploading */
+    size_t remaining;     /* Length of content */
+    size_t offset;        /* offset for reading */
+    /* Check whether the hdfsOpenFile has been called before */
+    int openFlag;
+    /* Whether to close the http connection for writing */
+    int closeFlag;
+    /* Synchronization between the curl and hdfsWrite threads */
+    pthread_mutex_t writeMutex;
+    /* 
+     * Transferring thread waits for this condition
+     * when there is no more content for transferring in the buffer
+     */
+    pthread_cond_t newwrite_or_close;
+    /* Condition used to indicate finishing transferring (one buffer) */
+    pthread_cond_t transfer_finish;
+};
 
 
+/** File handle for webhdfs */
 struct webhdfsFileHandle {
 struct webhdfsFileHandle {
-    char *absPath;
-    int bufferSize;
-    short replication;
-    tSize blockSize;
-    char *datanode;
-    webhdfsBuffer *uploadBuffer;
+    char *absPath;        /* Absolute path of file */
+    int bufferSize;       /* Size of buffer */
+    short replication;    /* Number of replication */
+    tSize blockSize;      /* Block size */
+    char *datanode;       /* URL of the DataNode */
+    /* webhdfsBuffer handle used to store the upload data */
+    struct webhdfsBuffer *uploadBuffer;
+    /* The thread used for data transferring */
     pthread_t connThread;
     pthread_t connThread;
 };
 };
 
 
+/** Type of http header */
 enum HttpHeader {
 enum HttpHeader {
     GET,
     GET,
     PUT,
     PUT,
@@ -65,44 +77,218 @@ enum HttpHeader {
     DELETE
     DELETE
 };
 };
 
 
+/** Whether to redirect */
 enum Redirect {
 enum Redirect {
     YES,
     YES,
     NO
     NO
 };
 };
 
 
-typedef struct {
+/** Buffer used for holding response */
+struct ResponseBuffer {
     char *content;
     char *content;
     size_t remaining;
     size_t remaining;
     size_t offset;
     size_t offset;
-} ResponseBufferInternal;
-typedef ResponseBufferInternal *ResponseBuffer;
+};
 
 
 /**
 /**
  * The response got through webhdfs
  * The response got through webhdfs
  */
  */
-typedef struct {
-    ResponseBuffer body;
-    ResponseBuffer header;
-}* Response;
-
-ResponseBuffer initResponseBuffer();
-void freeResponseBuffer(ResponseBuffer buffer);
-void freeResponse(Response resp);
-
-Response launchMKDIR(char *url);
-Response launchRENAME(char *url);
-Response launchCHMOD(char *url);
-Response launchGFS(char *url);
-Response launchLS(char *url);
-Response launchDELETE(char *url);
-Response launchCHOWN(char *url);
-Response launchOPEN(char *url, Response resp);
-Response launchUTIMES(char *url);
-Response launchNnWRITE(char *url);
-
-Response launchDnWRITE(const char *url, webhdfsBuffer *buffer);
-Response launchNnAPPEND(char *url);
-Response launchSETREPLICATION(char *url);
-Response launchDnAPPEND(const char *url, webhdfsBuffer *buffer);
+struct Response {
+    struct ResponseBuffer *body;
+    struct ResponseBuffer *header;
+};
+
+/**
+ * Create and initialize a ResponseBuffer
+ *
+ * @param buffer Pointer pointing to new created ResponseBuffer handle
+ * @return 0 for success, non-zero value to indicate error
+ */
+int initResponseBuffer(struct ResponseBuffer **buffer) __attribute__ ((warn_unused_result));
+
+/**
+ * Free the given ResponseBuffer
+ *
+ * @param buffer The ResponseBuffer to free
+ */
+void freeResponseBuffer(struct ResponseBuffer *buffer);
+
+/**
+ * Free the given Response
+ *
+ * @param resp The Response to free
+ */
+void freeResponse(struct Response *resp);
+
+/**
+ * Send the MKDIR request to NameNode using the given URL. 
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for MKDIR operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchMKDIR(const char *url,
+                struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the RENAME request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for RENAME operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchRENAME(const char *url,
+                 struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the CHMOD request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for CHMOD operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchCHMOD(const char *url,
+                struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the GetFileStatus request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for GetFileStatus operation
+ * @param response Response handle to store response returned from the NameNode,
+ *                 containing either file status or exception information
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchGFS(const char *url,
+              struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the LS (LISTSTATUS) request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for LISTSTATUS operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchLS(const char *url,
+             struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the DELETE request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for DELETE operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchDELETE(const char *url,
+                 struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the CHOWN request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for CHOWN operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchCHOWN(const char *url,
+                struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the OPEN request to NameNode using the given URL, 
+ * asking for reading a file (within a range). 
+ * The NameNode first redirects the request to the datanode
+ * that holds the corresponding first block of the file (within a range),
+ * and the datanode returns the content of the file through the HTTP connection.
+ *
+ * @param url The URL for OPEN operation
+ * @param resp The response holding user's buffer. 
+               The file content will be written into the buffer.
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchOPEN(const char *url,
+               struct Response* resp) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the SETTIMES request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for SETTIMES operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchUTIMES(const char *url,
+                 struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the WRITE/CREATE request to NameNode using the given URL.
+ * The NameNode will choose the writing target datanodes 
+ * and return the first datanode in the pipeline as response
+ *
+ * @param url The URL for WRITE/CREATE operation connecting to NameNode
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchNnWRITE(const char *url,
+                  struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the WRITE request along with to-write content to 
+ * the corresponding DataNode using the given URL. 
+ * The DataNode will write the data and return the response.
+ *
+ * @param url The URL for WRITE operation connecting to DataNode
+ * @param buffer The webhdfsBuffer containing data to be written to hdfs
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchDnWRITE(const char *url, struct webhdfsBuffer *buffer,
+                  struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the WRITE (APPEND) request to NameNode using the given URL. 
+ * The NameNode determines the DataNode for appending and 
+ * sends its URL back as response.
+ *
+ * @param url The URL for APPEND operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchNnAPPEND(const char *url, struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the SETREPLICATION request to NameNode using the given URL.
+ * The NameNode will execute the operation and return the result as response.
+ *
+ * @param url The URL for SETREPLICATION operation
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchSETREPLICATION(const char *url,
+                         struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Send the APPEND request along with the content to DataNode.
+ * The DataNode will do the appending and return the result as response.
+ *
+ * @param url The URL for APPEND operation connecting to DataNode
+ * @param buffer The webhdfsBuffer containing data to be appended
+ * @param response Response handle to store response returned from the NameNode
+ * @return 0 for success, non-zero value to indicate error
+ */
+int launchDnAPPEND(const char *url, struct webhdfsBuffer *buffer,
+                   struct Response **response) __attribute__ ((warn_unused_result));
+
+/**
+ * Call sys_errlist to get the error message string for the given error code
+ *
+ * @param errnoval  The error code value
+ * @return          The error message string mapped to the given error code
+ */
+const char *hdfs_strerror(int errnoval);
 
 
 #endif //_HDFS_HTTP_CLIENT_H_
 #endif //_HDFS_HTTP_CLIENT_H_

+ 333 - 185
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.c

@@ -22,233 +22,381 @@
 #include <stdio.h>
 #include <stdio.h>
 #include <errno.h>
 #include <errno.h>
 
 
-#define NUM_OF_PERMISSION_BITS 4
-#define NUM_OF_PORT_BITS 6
-#define NUM_OF_REPLICATION_BITS 6
-
-static char *prepareQUERY(const char *host, int nnPort, const char *srcpath, const char *OP, const char *user) {
-    size_t length;
-    char *url;
-    const char *const protocol = "http://";
-    const char *const prefix = "/webhdfs/v1";
-    char *temp;
-    char *port;
-    port= (char*) malloc(NUM_OF_PORT_BITS);
-    if (!port) {
-        return NULL;
-    }
-    sprintf(port,"%d",nnPort);
-    if (user != NULL) {
-        length = strlen(protocol) + strlen(host) + strlen(":") + strlen(port) + strlen(prefix) + strlen(srcpath) + strlen ("?op=") + strlen(OP) + strlen("&user.name=") + strlen(user);
-    } else {
-        length = strlen(protocol) + strlen(host) + strlen(":") + strlen(port) + strlen(prefix) + strlen(srcpath) +  strlen ("?op=") + strlen(OP);
-    }
-    
-    temp = (char*) malloc(length + 1);
-    if (!temp) {
-        return NULL;
-    }
-    strcpy(temp,protocol);
-    temp = strcat(temp,host);
-    temp = strcat(temp,":");
-    temp = strcat(temp,port);
-    temp = strcat(temp,prefix);
-    temp = strcat(temp,srcpath);
-    temp = strcat(temp,"?op=");
-    temp = strcat(temp,OP);
-    if (user) {
-        temp = strcat(temp,"&user.name=");
-        temp = strcat(temp,user);
-    }
-    url = temp;
-    return url;
-}
-
+#define PERM_STR_LEN 4  // "644" + one byte for NUL
+#define SHORT_STR_LEN 6 // 65535 + NUL
+#define LONG_STR_LEN 21 // 2^64-1 = 18446744073709551615 + NUL
 
 
-static int decToOctal(int decNo) {
-    int octNo=0;
-    int expo =0;
-    while (decNo != 0)  {
-        octNo = ((decNo % 8) * pow(10,expo)) + octNo;
-        decNo = decNo / 8;
-        expo++;
+/**
+ * Create query based on NameNode hostname,
+ * NameNode port, path, operation and other parameters
+ *
+ * @param host          NameNode hostName
+ * @param nnPort        Port of NameNode
+ * @param path          Absolute path for the corresponding file
+ * @param op            Operations
+ * @param paraNum       Number of remaining parameters
+ * @param paraNames     Names of remaining parameters
+ * @param paraValues    Values of remaining parameters
+ * @param url           Holding the created URL
+ * @return 0 on success and non-zero value to indicate error
+ */
+static int createQueryURL(const char *host, unsigned int nnPort,
+                          const char *path, const char *op, int paraNum,
+                          const char **paraNames, const char **paraValues,
+                          char **queryUrl)
+{
+    size_t length = 0;
+    int i = 0, offset = 0, ret = 0;
+    char *url = NULL;
+    const char *protocol = "http://";
+    const char *prefix = "/webhdfs/v1";
+    
+    if (!paraNames || !paraValues) {
+        return EINVAL;
+    }
+    length = strlen(protocol) + strlen(host) + strlen(":") +
+                SHORT_STR_LEN + strlen(prefix) + strlen(path) +
+                strlen ("?op=") + strlen(op);
+    for (i = 0; i < paraNum; i++) {
+        if (paraNames[i] && paraValues[i]) {
+            length += 2 + strlen(paraNames[i]) + strlen(paraValues[i]);
+        }
+    }
+    url = malloc(length);   // The '\0' has already been included
+                            // when using SHORT_STR_LEN
+    if (!url) {
+        return ENOMEM;
+    }
+    
+    offset = snprintf(url, length, "%s%s:%d%s%s?op=%s",
+                      protocol, host, nnPort, prefix, path, op);
+    if (offset >= length || offset < 0) {
+        ret = EIO;
+        goto done;
+    }
+    for (i = 0; i < paraNum; i++) {
+        if (!paraNames[i] || !paraValues[i] || paraNames[i][0] == '\0' ||
+            paraValues[i][0] == '\0') {
+            continue;
+        }
+        offset += snprintf(url + offset, length - offset,
+                           "&%s=%s", paraNames[i], paraValues[i]);
+        if (offset >= length || offset < 0) {
+            ret = EIO;
+            goto done;
+        }
     }
     }
-    return octNo;
+done:
+    if (ret) {
+        free(url);
+        return ret;
+    }
+    *queryUrl = url;
+    return 0;
 }
 }
 
 
-
-char *prepareMKDIR(const char *host, int nnPort, const char *dirsubpath, const char *user) {
-    return prepareQUERY(host, nnPort, dirsubpath, "MKDIRS", user);
+int createUrlForMKDIR(const char *host, int nnPort,
+                      const char *path, const char *user, char **url)
+{
+    const char *userPara = "user.name";
+    return createQueryURL(host, nnPort, path, "MKDIRS", 1,
+                          &userPara, &user, url);
 }
 }
 
 
-
-char *prepareMKDIRwithMode(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user) {
-    char *url;
-    char *permission;
-    permission = (char*) malloc(NUM_OF_PERMISSION_BITS);
-    if (!permission) {
-        return NULL;
-    }
-    mode = decToOctal(mode);
-    sprintf(permission,"%d",mode);
-    url = prepareMKDIR(host, nnPort, dirsubpath, user);
-    url = realloc(url,(strlen(url) + strlen("&permission=") + strlen(permission) + 1));
-    if (!url) {
-        return NULL;
-    }
-    url = strcat(url,"&permission=");
-    url = strcat(url,permission);
-    return url;
+int createUrlForGetFileStatus(const char *host, int nnPort, const char *path,
+                              const char *user, char **url)
+{
+    const char *userPara = "user.name";
+    return createQueryURL(host, nnPort, path, "GETFILESTATUS", 1,
+                          &userPara, &user, url);
 }
 }
 
 
+int createUrlForLS(const char *host, int nnPort, const char *path,
+                   const char *user, char **url)
+{
+    const char *userPara = "user.name";
+    return createQueryURL(host, nnPort, path, "LISTSTATUS",
+                          1, &userPara, &user, url);
+}
 
 
-char *prepareRENAME(const char *host, int nnPort, const char *srcpath, const char *destpath, const char *user) {
-    char *url;
-    url = prepareQUERY(host, nnPort, srcpath, "RENAME", user);
-    url = realloc(url,(strlen(url) + strlen("&destination=") + strlen(destpath) + 1));
-    if (!url) {
-        return NULL;
-    }
-    url = strcat(url,"&destination=");
-    url = strcat(url,destpath);
-    return url;
+int createUrlForNnAPPEND(const char *host, int nnPort, const char *path,
+                         const char *user, char **url)
+{
+    const char *userPara = "user.name";
+    return createQueryURL(host, nnPort, path, "APPEND",
+                          1, &userPara, &user, url);
 }
 }
 
 
-char *prepareGFS(const char *host, int nnPort, const char *dirsubpath, const char *user) {
-    return (prepareQUERY(host, nnPort, dirsubpath, "GETFILESTATUS", user));
+int createUrlForMKDIRwithMode(const char *host, int nnPort, const char *path,
+                              int mode, const char *user, char **url)
+{
+    int strlength;
+    char permission[PERM_STR_LEN];
+    const char *paraNames[2], *paraValues[2];
+    
+    paraNames[0] = "permission";
+    paraNames[1] = "user.name";
+    memset(permission, 0, PERM_STR_LEN);
+    strlength = snprintf(permission, PERM_STR_LEN, "%o", mode);
+    if (strlength < 0 || strlength >= PERM_STR_LEN) {
+        return EIO;
+    }
+    paraValues[0] = permission;
+    paraValues[1] = user;
+    
+    return createQueryURL(host, nnPort, path, "MKDIRS", 2,
+                          paraNames, paraValues, url);
 }
 }
 
 
-char *prepareLS(const char *host, int nnPort, const char *dirsubpath, const char *user) {
-    return (prepareQUERY(host, nnPort, dirsubpath, "LISTSTATUS", user));
+int createUrlForRENAME(const char *host, int nnPort, const char *srcpath,
+                         const char *destpath, const char *user, char **url)
+{
+    const char *paraNames[2], *paraValues[2];
+    paraNames[0] = "destination";
+    paraNames[1] = "user.name";
+    paraValues[0] = destpath;
+    paraValues[1] = user;
+    
+    return createQueryURL(host, nnPort, srcpath,
+                          "RENAME", 2, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareCHMOD(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user) {
-    char *url;
-    char *permission;
-    permission = (char*) malloc(NUM_OF_PERMISSION_BITS);
-    if (!permission) {
-        return NULL;
-    }
-    mode &= 0x3FFF;
-    mode = decToOctal(mode);
-    sprintf(permission,"%d",mode);
-    url = prepareQUERY(host, nnPort, dirsubpath, "SETPERMISSION", user);
-    url = realloc(url,(strlen(url) + strlen("&permission=") + strlen(permission) + 1));
-    if (!url) {
-        return NULL;
+int createUrlForCHMOD(const char *host, int nnPort, const char *path,
+                      int mode, const char *user, char **url)
+{
+    int strlength;
+    char permission[PERM_STR_LEN];
+    const char *paraNames[2], *paraValues[2];
+    
+    paraNames[0] = "permission";
+    paraNames[1] = "user.name";
+    memset(permission, 0, PERM_STR_LEN);
+    strlength = snprintf(permission, PERM_STR_LEN, "%o", mode);
+    if (strlength < 0 || strlength >= PERM_STR_LEN) {
+        return EIO;
     }
     }
-    url = strcat(url,"&permission=");
-    url = strcat(url,permission);
-    return url;
+    paraValues[0] = permission;
+    paraValues[1] = user;
+    
+    return createQueryURL(host, nnPort, path, "SETPERMISSION",
+                          2, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareDELETE(const char *host, int nnPort, const char *dirsubpath, int recursive, const char *user) {
-    char *url = (prepareQUERY(host, nnPort, dirsubpath, "DELETE", user));
-    char *recursiveFlag = (char *)malloc(6);
-    if (!recursive) {
-        strcpy(recursiveFlag, "false");
+int createUrlForDELETE(const char *host, int nnPort, const char *path,
+                       int recursive, const char *user, char **url)
+{
+    const char *paraNames[2], *paraValues[2];
+    paraNames[0] = "recursive";
+    paraNames[1] = "user.name";
+    if (recursive) {
+        paraValues[0] = "true";
     } else {
     } else {
-        strcpy(recursiveFlag, "true");
-    }
-    url = (char *) realloc(url, strlen(url) + strlen("&recursive=") + strlen(recursiveFlag) + 1);
-    if (!url) {
-        return NULL;
+        paraValues[0] = "false";
     }
     }
+    paraValues[1] = user;
     
     
-    strcat(url, "&recursive=");
-    strcat(url, recursiveFlag);
-    return url;
+    return createQueryURL(host, nnPort, path, "DELETE",
+                          2, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareCHOWN(const char *host, int nnPort, const char *dirsubpath, const char *owner, const char *group, const char *user) {
-    char *url;
-    url = prepareQUERY(host, nnPort, dirsubpath, "SETOWNER", user);
-    if (!url) {
-        return NULL;
-    }
-    if(owner != NULL) {
-        url = realloc(url,(strlen(url) + strlen("&owner=") + strlen(owner) + 1));
-        url = strcat(url,"&owner=");
-        url = strcat(url,owner);
-    }
-    if (group != NULL) {
-        url = realloc(url,(strlen(url) + strlen("&group=") + strlen(group) + 1));
-        url = strcat(url,"&group=");
-        url = strcat(url,group);
-    }
-    return url;
+int createUrlForCHOWN(const char *host, int nnPort, const char *path,
+                      const char *owner, const char *group,
+                      const char *user, char **url)
+{
+    const char *paraNames[3], *paraValues[3];
+    paraNames[0] = "owner";
+    paraNames[1] = "group";
+    paraNames[2] = "user.name";
+    paraValues[0] = owner;
+    paraValues[1] = group;
+    paraValues[2] = user;
+    
+    return createQueryURL(host, nnPort, path, "SETOWNER",
+                          3, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareOPEN(const char *host, int nnPort, const char *dirsubpath, const char *user, size_t offset, size_t length) {
-    char *base_url = prepareQUERY(host, nnPort, dirsubpath, "OPEN", user);
-    char *url = (char *) malloc(strlen(base_url) + strlen("&offset=") + 15 + strlen("&length=") + 15);
-    if (!url) {
-        return NULL;
+int createUrlForOPEN(const char *host, int nnPort, const char *path,
+                     const char *user, size_t offset, size_t length, char **url)
+{
+    int strlength;
+    char offsetStr[LONG_STR_LEN], lengthStr[LONG_STR_LEN];
+    const char *paraNames[3], *paraValues[3];
+    
+    paraNames[0] = "offset";
+    paraNames[1] = "length";
+    paraNames[2] = "user.name";
+    memset(offsetStr, 0, LONG_STR_LEN);
+    memset(lengthStr, 0, LONG_STR_LEN);
+    strlength = snprintf(offsetStr, LONG_STR_LEN, "%lu", offset);
+    if (strlength < 0 || strlength >= LONG_STR_LEN) {
+        return EIO;
     }
     }
-    sprintf(url, "%s&offset=%ld&length=%ld", base_url, offset, length);
-    return url;
+    strlength = snprintf(lengthStr, LONG_STR_LEN, "%lu", length);
+    if (strlength < 0 || strlength >= LONG_STR_LEN) {
+        return EIO;
+    }
+    paraValues[0] = offsetStr;
+    paraValues[1] = lengthStr;
+    paraValues[2] = user;
+    
+    return createQueryURL(host, nnPort, path, "OPEN",
+                          3, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareUTIMES(const char *host, int nnPort, const char *dirsubpath, long unsigned mTime, long unsigned aTime, const char *user) {
-    char *url;
-    char *modTime;
-    char *acsTime;
-    modTime = (char*) malloc(12);
-    acsTime = (char*) malloc(12);
-    url = prepareQUERY(host, nnPort, dirsubpath, "SETTIMES", user);
-    sprintf(modTime,"%lu",mTime);
-    sprintf(acsTime,"%lu",aTime);
-    url = realloc(url,(strlen(url) + strlen("&modificationtime=") + strlen(modTime) + strlen("&accesstime=") + strlen(acsTime) + 1));
-    if (!url) {
-        return NULL;
+int createUrlForUTIMES(const char *host, int nnPort, const char *path,
+                       long unsigned mTime, long unsigned aTime,
+                       const char *user, char **url)
+{
+    int strlength;
+    char modTime[LONG_STR_LEN], acsTime[LONG_STR_LEN];
+    const char *paraNames[3], *paraValues[3];
+    
+    memset(modTime, 0, LONG_STR_LEN);
+    memset(acsTime, 0, LONG_STR_LEN);
+    strlength = snprintf(modTime, LONG_STR_LEN, "%lu", mTime);
+    if (strlength < 0 || strlength >= LONG_STR_LEN) {
+        return EIO;
     }
     }
-    url = strcat(url, "&modificationtime=");
-    url = strcat(url, modTime);
-    url = strcat(url,"&accesstime=");
-    url = strcat(url, acsTime);
-    return url;
+    strlength = snprintf(acsTime, LONG_STR_LEN, "%lu", aTime);
+    if (strlength < 0 || strlength >= LONG_STR_LEN) {
+        return EIO;
+    }
+    paraNames[0] = "modificationtime";
+    paraNames[1] = "accesstime";
+    paraNames[2] = "user.name";
+    paraValues[0] = modTime;
+    paraValues[1] = acsTime;
+    paraValues[2] = user;
+    
+    return createQueryURL(host, nnPort, path, "SETTIMES",
+                          3, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareNnWRITE(const char *host, int nnPort, const char *dirsubpath, const char *user, int16_t replication, size_t blockSize) {
-    char *url;
-    url = prepareQUERY(host, nnPort, dirsubpath, "CREATE", user);
-    url = realloc(url, (strlen(url) + strlen("&overwrite=true") + 1));
-    if (!url) {
-        return NULL;
-    }
-    url = strcat(url, "&overwrite=true");
+int createUrlForNnWRITE(const char *host, int nnPort,
+                        const char *path, const char *user,
+                        int16_t replication, size_t blockSize, char **url)
+{
+    int strlength;
+    char repStr[SHORT_STR_LEN], blockSizeStr[LONG_STR_LEN];
+    const char *paraNames[4], *paraValues[4];
+    
+    memset(repStr, 0, SHORT_STR_LEN);
+    memset(blockSizeStr, 0, LONG_STR_LEN);
     if (replication > 0) {
     if (replication > 0) {
-        url = realloc(url, (strlen(url) + strlen("&replication=") + 6));
-        if (!url) {
-            return NULL;
+        strlength = snprintf(repStr, SHORT_STR_LEN, "%u", replication);
+        if (strlength < 0 || strlength >= SHORT_STR_LEN) {
+            return EIO;
         }
         }
-        sprintf(url, "%s&replication=%d", url, replication);
     }
     }
     if (blockSize > 0) {
     if (blockSize > 0) {
-        url = realloc(url, (strlen(url) + strlen("&blocksize=") + 16));
-        if (!url) {
-            return NULL;
+        strlength = snprintf(blockSizeStr, LONG_STR_LEN, "%lu", blockSize);
+        if (strlength < 0 || strlength >= LONG_STR_LEN) {
+            return EIO;
+        }
+    }
+    paraNames[0] = "overwrite";
+    paraNames[1] = "replication";
+    paraNames[2] = "blocksize";
+    paraNames[3] = "user.name";
+    paraValues[0] = "true";
+    paraValues[1] = repStr;
+    paraValues[2] = blockSizeStr;
+    paraValues[3] = user;
+    
+    return createQueryURL(host, nnPort, path, "CREATE",
+                          4, paraNames, paraValues, url);
+}
+
+int createUrlForSETREPLICATION(const char *host, int nnPort,
+                               const char *path, int16_t replication,
+                               const char *user, char **url)
+{
+    char repStr[SHORT_STR_LEN];
+    const char *paraNames[2], *paraValues[2];
+    int strlength;
+
+    memset(repStr, 0, SHORT_STR_LEN);
+    if (replication > 0) {
+        strlength = snprintf(repStr, SHORT_STR_LEN, "%u", replication);
+        if (strlength < 0 || strlength >= SHORT_STR_LEN) {
+            return EIO;
         }
         }
-        sprintf(url, "%s&blocksize=%ld", url, blockSize);
     }
     }
-    return url;
+    paraNames[0] = "replication";
+    paraNames[1] = "user.name";
+    paraValues[0] = repStr;
+    paraValues[1] = user;
+    
+    return createQueryURL(host, nnPort, path, "SETREPLICATION",
+                          2, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareNnAPPEND(const char *host, int nnPort, const char *dirsubpath, const char *user) {
-    return (prepareQUERY(host, nnPort, dirsubpath, "APPEND", user));
+int createUrlForGetBlockLocations(const char *host, int nnPort,
+                                  const char *path, size_t offset,
+                                  size_t length, const char *user, char **url)
+{
+    char offsetStr[LONG_STR_LEN], lengthStr[LONG_STR_LEN];
+    const char *paraNames[3], *paraValues[3];
+    int strlength;
+    
+    memset(offsetStr, 0, LONG_STR_LEN);
+    memset(lengthStr, 0, LONG_STR_LEN);
+    if (offset > 0) {
+        strlength = snprintf(offsetStr, LONG_STR_LEN, "%lu", offset);
+        if (strlength < 0 || strlength >= LONG_STR_LEN) {
+            return EIO;
+        }
+    }
+    if (length > 0) {
+        strlength = snprintf(lengthStr, LONG_STR_LEN, "%lu", length);
+        if (strlength < 0 || strlength >= LONG_STR_LEN) {
+            return EIO;
+        }
+    }
+    paraNames[0] = "offset";
+    paraNames[1] = "length";
+    paraNames[2] = "user.name";
+    paraValues[0] = offsetStr;
+    paraValues[1] = lengthStr;
+    paraValues[2] = user;
+    
+    return createQueryURL(host, nnPort, path, "GET_BLOCK_LOCATIONS",
+                          3, paraNames, paraValues, url);
 }
 }
 
 
-char *prepareSETREPLICATION(const char *host, int nnPort, const char *path, int16_t replication, const char *user)
+int createUrlForReadFromDatanode(const char *dnHost, int dnPort,
+                                 const char *path, size_t offset,
+                                 size_t length, const char *user,
+                                 const char *namenodeRpcAddr, char **url)
 {
 {
-    char *url = prepareQUERY(host, nnPort, path, "SETREPLICATION", user);
-    char *replicationNum = (char *) malloc(NUM_OF_REPLICATION_BITS);
-    sprintf(replicationNum, "%u", replication);
-    url = realloc(url, strlen(url) + strlen("&replication=") + strlen(replicationNum)+ 1);
-    if (!url) {
-        return NULL;
+    char offsetStr[LONG_STR_LEN], lengthStr[LONG_STR_LEN];
+    const char *paraNames[4], *paraValues[4];
+    int strlength;
+    
+    memset(offsetStr, 0, LONG_STR_LEN);
+    memset(lengthStr, 0, LONG_STR_LEN);
+    if (offset > 0) {
+        strlength = snprintf(offsetStr, LONG_STR_LEN, "%lu", offset);
+        if (strlength < 0 || strlength >= LONG_STR_LEN) {
+            return EIO;
+        }
     }
     }
+    if (length > 0) {
+        strlength = snprintf(lengthStr, LONG_STR_LEN, "%lu", length);
+        if (strlength < 0 || strlength >= LONG_STR_LEN) {
+            return EIO;
+        }
+    }
+    
+    paraNames[0] = "offset";
+    paraNames[1] = "length";
+    paraNames[2] = "user.name";
+    paraNames[3] = "namenoderpcaddress";
+    paraValues[0] = offsetStr;
+    paraValues[1] = lengthStr;
+    paraValues[2] = user;
+    paraValues[3] = namenodeRpcAddr;
     
     
-    url = strcat(url, "&replication=");
-    url = strcat(url, replicationNum);
-    return url;
+    return createQueryURL(dnHost, dnPort, path, "OPEN",
+                          4, paraNames, paraValues, url);
 }
 }

+ 215 - 16
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_http_query.h

@@ -20,22 +20,221 @@
 #ifndef _HDFS_HTTP_QUERY_H_
 #ifndef _HDFS_HTTP_QUERY_H_
 #define _HDFS_HTTP_QUERY_H_
 #define _HDFS_HTTP_QUERY_H_
 
 
-#include <stdint.h>
-#include <stdio.h>
-
-char *prepareMKDIR(const char *host, int nnPort, const char *dirsubpath, const char *user);
-char *prepareMKDIRwithMode(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user);
-char *prepareRENAME(const char *host, int nnPort, const char *srcpath, const char *destpath, const char *user);
-char *prepareCHMOD(const char *host, int nnPort, const char *dirsubpath, int mode, const char *user);
-char *prepareGFS(const char *host, int nnPort, const char *dirsubpath, const char *user);
-char *prepareLS(const char *host, int nnPort, const char *dirsubpath, const char *user);
-char *prepareDELETE(const char *host, int nnPort, const char *dirsubpath, int recursive, const char *user);
-char *prepareCHOWN(const char *host, int nnPort, const char *dirsubpath, const char *owner, const char *group, const char *user);
-char *prepareOPEN(const char *host, int nnPort, const char *dirsubpath, const char *user, size_t offset, size_t length);
-char *prepareUTIMES(const char *host, int nnPort, const char *dirsubpath, long unsigned mTime, long unsigned aTime, const char *user);
-char *prepareNnWRITE(const char *host, int nnPort, const char *dirsubpath, const char *user, int16_t replication, size_t blockSize);
-char *prepareNnAPPEND(const char *host, int nnPort, const char *dirsubpath, const char *user);
-char *prepareSETREPLICATION(const char *host, int nnPort, const char *path, int16_t replication, const char *user);
+#include <unistd.h> /* for size_t */
+#include <inttypes.h> /* for int16_t */
+
+/**
+ * Create the URL for a MKDIR request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the dir to create
+ * @param user User name
+ * @param url Holding the generated URL for MKDIR request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForMKDIR(const char *host, int nnPort,
+                      const char *path, const char *user,
+                      char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a MKDIR (with mode) request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the dir to create
+ * @param mode Mode of MKDIR
+ * @param user User name
+ * @param url Holding the generated URL for MKDIR request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForMKDIRwithMode(const char *host, int nnPort, const char *path,
+                              int mode, const char *user,
+                              char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a RENAME request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param srcpath Source path
+ * @param dstpath Destination path
+ * @param user User name
+ * @param url Holding the generated URL for RENAME request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForRENAME(const char *host, int nnPort, const char *srcpath,
+                       const char *dstpath, const char *user,
+                       char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a CHMOD request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Target path
+ * @param mode New mode for the file
+ * @param user User name
+ * @param url Holding the generated URL for CHMOD request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForCHMOD(const char *host, int nnPort, const char *path,
+                      int mode, const char *user,
+                      char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a GETFILESTATUS request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the target file
+ * @param user User name
+ * @param url Holding the generated URL for GETFILESTATUS request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForGetFileStatus(const char *host, int nnPort,
+                              const char *path, const char *user,
+                              char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a LISTSTATUS request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the directory for listing
+ * @param user User name
+ * @param url Holding the generated URL for LISTSTATUS request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForLS(const char *host, int nnPort,
+                   const char *path, const char *user,
+                   char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a DELETE request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the file to be deletected
+ * @param recursive Whether or not to delete in a recursive way
+ * @param user User name
+ * @param url Holding the generated URL for DELETE request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForDELETE(const char *host, int nnPort, const char *path,
+                       int recursive, const char *user,
+                       char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a CHOWN request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the target
+ * @param owner New owner
+ * @param group New group
+ * @param user User name
+ * @param url Holding the generated URL for CHOWN request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForCHOWN(const char *host, int nnPort, const char *path,
+                      const char *owner, const char *group, const char *user,
+                      char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a OPEN/READ request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the file to read
+ * @param user User name
+ * @param offset Offset for reading (the start position for this read)
+ * @param length Length of the file to read
+ * @param url Holding the generated URL for OPEN/READ request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForOPEN(const char *host, int nnPort, const char *path,
+                     const char *user, size_t offset, size_t length,
+                     char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a UTIMES (update time) request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the file for updating time
+ * @param mTime Modified time to set
+ * @param aTime Access time to set
+ * @param user User name
+ * @param url Holding the generated URL for UTIMES request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForUTIMES(const char *host, int nnPort, const char *path,
+                       long unsigned mTime, long unsigned aTime,
+                       const char *user,
+                       char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a WRITE/CREATE request (sent to NameNode)
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the dir to create
+ * @param user User name
+ * @param replication Number of replication of the file
+ * @param blockSize Size of the block for the file
+ * @param url Holding the generated URL for WRITE request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForNnWRITE(const char *host, int nnPort, const char *path,
+                        const char *user, int16_t replication, size_t blockSize,
+                        char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for an APPEND request (sent to NameNode)
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the file for appending
+ * @param user User name
+ * @param url Holding the generated URL for APPEND request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForNnAPPEND(const char *host, int nnPort,
+                         const char *path, const char *user,
+                         char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a SETREPLICATION request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the target file
+ * @param replication New replication number
+ * @param user User name
+ * @param url Holding the generated URL for SETREPLICATION request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForSETREPLICATION(const char *host, int nnPort, const char *path,
+                               int16_t replication, const char *user,
+                               char **url) __attribute__ ((warn_unused_result));
+
+/**
+ * Create the URL for a GET_BLOCK_LOCATIONS request
+ *
+ * @param host The hostname of the NameNode
+ * @param nnPort Port of the NameNode
+ * @param path Path of the target file
+ * @param offset The offset in the file
+ * @param length Length of the file content
+ * @param user User name
+ * @param url Holding the generated URL for GET_BLOCK_LOCATIONS request
+ * @return 0 on success and non-zero value on errors
+ */
+int createUrlForGetBlockLocations(const char *host, int nnPort,
+                            const char *path, size_t offset,
+                            size_t length, const char *user,
+                            char **url) __attribute__ ((warn_unused_result));
 
 
 
 
 #endif  //_HDFS_HTTP_QUERY_H_
 #endif  //_HDFS_HTTP_QUERY_H_

+ 478 - 267
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.c

@@ -25,6 +25,11 @@
 #include <ctype.h>
 #include <ctype.h>
 #include <jansson.h>
 #include <jansson.h>
 
 
+static const char * const temporaryRedirectCode = "307 TEMPORARY_REDIRECT";
+static const char * const twoHundredOKCode = "200 OK";
+static const char * const twoHundredOneCreatedCode = "201 Created";
+static const char * const httpHeaderString = "HTTP/1.1";
+
 /**
 /**
  * Exception information after calling JSON operations
  * Exception information after calling JSON operations
  */
  */
@@ -34,9 +39,6 @@ struct jsonException {
   const char *message;
   const char *message;
 };
 };
 
 
-static hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat,
-                           int *numEntries, const char *operation);
-
 static void dotsToSlashes(char *str)
 static void dotsToSlashes(char *str)
 {
 {
     for (; *str != '\0'; str++) {
     for (; *str != '\0'; str++) {
@@ -45,8 +47,9 @@ static void dotsToSlashes(char *str)
     }
     }
 }
 }
 
 
-int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
-                        const char *fmt, va_list ap)
+/** Print out the JSON exception information */
+static int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
+                               const char *fmt, va_list ap)
 {
 {
     char *javaClassName = NULL;
     char *javaClassName = NULL;
     int excErrno = EINTERNAL, shouldPrint = 0;
     int excErrno = EINTERNAL, shouldPrint = 0;
@@ -74,11 +77,23 @@ int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
     return excErrno;
     return excErrno;
 }
 }
 
 
-int printJsonException(struct jsonException *exc, int noPrintFlags,
-                       const char *fmt, ...)
+/**
+ * Print out JSON exception information.
+ *
+ * @param exc             The exception information to print and free
+ * @param noPrintFlags    Flags which determine which exceptions we should NOT
+ *                        print.
+ * @param fmt             Printf-style format list
+ * @param ...             Printf-style varargs
+ *
+ * @return                The POSIX error number associated with the exception
+ *                        object.
+ */
+static int printJsonException(struct jsonException *exc, int noPrintFlags,
+                              const char *fmt, ...)
 {
 {
     va_list ap;
     va_list ap;
-    int ret;
+    int ret = 0;
     
     
     va_start(ap, fmt);
     va_start(ap, fmt);
     ret = printJsonExceptionV(exc, noPrintFlags, fmt, ap);
     ret = printJsonExceptionV(exc, noPrintFlags, fmt, ap);
@@ -86,81 +101,20 @@ int printJsonException(struct jsonException *exc, int noPrintFlags,
     return ret;
     return ret;
 }
 }
 
 
-static hdfsFileInfo *json_parse_array(json_t *jobj, char *key, hdfsFileInfo *fileStat, int *numEntries, const char *operation) {
-    int arraylen = json_array_size(jobj);                      //Getting the length of the array
-    *numEntries = arraylen;
-    if (!key) {
-        return NULL;
-    }
-    if(arraylen > 0) {
-        fileStat = (hdfsFileInfo *)realloc(fileStat,sizeof(hdfsFileInfo)*arraylen);
-    }
-    json_t *jvalue;
-    int i;
-    for (i=0; i< arraylen; i++) {
-        jvalue = json_array_get(jobj, i);            //Getting the array element at position i
-        if (json_is_array(jvalue)) {                 // array within an array - program should never come here for now
-            json_parse_array(jvalue, NULL, &fileStat[i], numEntries, operation);
-        }
-        else if (json_is_object(jvalue)) {           // program will definitely come over here
-            parseJsonGFS(jvalue, &fileStat[i], numEntries, operation);
-        }
-        else {
-            return NULL;                               // program will never come over here for now
-        }
-    }
-    *numEntries = arraylen;
-    return fileStat;
-}
-
-int parseBoolean(char *response) {
-    json_t *root;
-    json_error_t error;
-    size_t flags = 0;
-    int result = 0;
-    const char *key;
-    json_t *value;
-    root = json_loads(response, flags, &error);
-    void *iter = json_object_iter(root);
-    while(iter)  {
-        key = json_object_iter_key(iter);
-        value = json_object_iter_value(iter);
-        switch (json_typeof(value))  {
-            case JSON_TRUE:
-                result = 1;
-                break;
-            default:
-                result = 0;
-                break;
-        }
-        iter = json_object_iter_next(root, iter);
-    }
-    return result;
-}
-
-int parseMKDIR(char *response) {
-    return (parseBoolean(response));
-}
-
-int parseRENAME(char *response) {
-    return (parseBoolean(response));
-}
-
-int parseDELETE(char *response) {
-    return (parseBoolean(response));
-}
-
-struct jsonException *parseJsonException(json_t *jobj) {
-    const char *key;
-    json_t *value;
+/** Parse the exception information from JSON */
+static struct jsonException *parseJsonException(json_t *jobj)
+{
+    const char *key = NULL;
+    json_t *value = NULL;
     struct jsonException *exception = NULL;
     struct jsonException *exception = NULL;
+    void *iter = NULL;
     
     
     exception = calloc(1, sizeof(*exception));
     exception = calloc(1, sizeof(*exception));
     if (!exception) {
     if (!exception) {
         return NULL;
         return NULL;
     }
     }
     
     
-    void *iter = json_object_iter(jobj);
+    iter = json_object_iter(jobj);
     while (iter) {
     while (iter) {
         key = json_object_iter_key(iter);
         key = json_object_iter_key(iter);
         value = json_object_iter_value(iter);
         value = json_object_iter_value(iter);
@@ -175,23 +129,31 @@ struct jsonException *parseJsonException(json_t *jobj) {
         
         
         iter = json_object_iter_next(jobj, iter);
         iter = json_object_iter_next(jobj, iter);
     }
     }
-    
     return exception;
     return exception;
 }
 }
 
 
-struct jsonException *parseException(const char *content) {
-    if (!content) {
-        return NULL;
-    }
-    
+/** 
+ * Parse the exception information which is presented in JSON
+ * 
+ * @param content   Exception information in JSON
+ * @return          jsonException for printing out
+ */
+static struct jsonException *parseException(const char *content)
+{
     json_error_t error;
     json_error_t error;
     size_t flags = 0;
     size_t flags = 0;
-    const char *key;
+    const char *key = NULL;
     json_t *value;
     json_t *value;
-    json_t *jobj = json_loads(content, flags, &error);
+    json_t *jobj;
+    struct jsonException *exception = NULL;
     
     
+    if (!content) {
+        return NULL;
+    }
+    jobj = json_loads(content, flags, &error);
     if (!jobj) {
     if (!jobj) {
-        fprintf(stderr, "JSon parsing failed\n");
+        fprintf(stderr, "JSon parsing error: on line %d: %s\n",
+                error.line, error.text);
         return NULL;
         return NULL;
     }
     }
     void *iter = json_object_iter(jobj);
     void *iter = json_object_iter(jobj);
@@ -199,254 +161,503 @@ struct jsonException *parseException(const char *content) {
         key = json_object_iter_key(iter);
         key = json_object_iter_key(iter);
         value = json_object_iter_value(iter);
         value = json_object_iter_value(iter);
         
         
-        if (!strcmp(key, "RemoteException") && json_typeof(value) == JSON_OBJECT) {
-            return parseJsonException(value);
+        if (!strcmp(key, "RemoteException") &&
+                    json_typeof(value) == JSON_OBJECT) {
+            exception = parseJsonException(value);
+            break;
         }
         }
         iter = json_object_iter_next(jobj, iter);
         iter = json_object_iter_next(jobj, iter);
     }
     }
-    return NULL;
+    
+    json_decref(jobj);
+    return exception;
 }
 }
 
 
-static hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat,
-                                  int *numEntries, const char *operation)
+/**
+ * Parse the response information which uses TRUE/FALSE 
+ * to indicate whether the operation succeeded
+ *
+ * @param response  Response information
+ * @return          0 to indicate success
+ */
+static int parseBoolean(const char *response)
 {
 {
-    const char *tempstr;
-    const char *key;
-    json_t *value;
-    void *iter = json_object_iter(jobj);
-    while(iter)  {
-        key = json_object_iter_key(iter);
-        value = json_object_iter_value(iter);
-        
-        switch (json_typeof(value)) {
-            case JSON_INTEGER:
-                if(!strcmp(key,"accessTime")) {
-                    fileStat->mLastAccess = (time_t)(json_integer_value(value)/1000);
-                } else if (!strcmp(key,"blockSize")) {
-                    fileStat->mBlockSize = (tOffset)json_integer_value(value);
-                } else if (!strcmp(key,"length")) {
-                    fileStat->mSize = (tOffset)json_integer_value(value);
-                } else if(!strcmp(key,"modificationTime")) {
-                    fileStat->mLastMod = (time_t)(json_integer_value(value)/1000);
-                } else if (!strcmp(key,"replication")) {
-                    fileStat->mReplication = (short)json_integer_value(value);
-                }
-                break;
-                
-            case JSON_STRING:
-                if(!strcmp(key,"group")) {
-                    fileStat->mGroup=(char *)json_string_value(value);
-                } else if (!strcmp(key,"owner")) {
-                    fileStat->mOwner=(char *)json_string_value(value);
-                } else if (!strcmp(key,"pathSuffix")) {
-                    fileStat->mName=(char *)json_string_value(value);
-                } else if (!strcmp(key,"permission")) {
-                    tempstr=(char *)json_string_value(value);
-                    fileStat->mPermissions = (short)strtol(tempstr,(char **)NULL,8);
-                } else if (!strcmp(key,"type")) {
-                    char *cvalue = (char *)json_string_value(value);
-                    if (!strcmp(cvalue, "DIRECTORY")) {
-                        fileStat->mKind = kObjectKindDirectory;
-                    } else {
-                        fileStat->mKind = kObjectKindFile;
-                    }
-                }
-                break;
-                
-            case JSON_OBJECT:
-                if(!strcmp(key,"FileStatus")) {
-                    parseJsonGFS(value, fileStat, numEntries, operation);
-                } else if (!strcmp(key,"FileStatuses")) {
-                    fileStat = parseJsonGFS(value, &fileStat[0], numEntries, operation);
-                } else if (!strcmp(key,"RemoteException")) {
-                    //Besides returning NULL, we also need to print the exception information
-                    struct jsonException *exception = parseJsonException(value);
-                    if (exception) {
-                        errno = printJsonException(exception, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
-                    }
-                    
-                    if(fileStat != NULL) {
-                        free(fileStat);
-                        fileStat = NULL;
-                    }
-                }
-                break;
-                
-            case JSON_ARRAY:
-                if (!strcmp(key,"FileStatus")) {
-                    fileStat = json_parse_array(value,(char *) key,fileStat,numEntries, operation);
-                }
-                break;
-                
-            default:
-                if(fileStat != NULL) {
-                    free(fileStat);
-                    fileStat = NULL;
-                }
-        }
-        iter = json_object_iter_next(jobj, iter);
+    json_t *root, *value;
+    json_error_t error;
+    size_t flags = 0;
+    int result = 0;
+    
+    root = json_loads(response, flags, &error);
+    if (!root) {
+        fprintf(stderr, "JSon parsing error: on line %d: %s\n",
+                error.line, error.text);
+        return EIO;
+    }
+    void *iter = json_object_iter(root);
+    value = json_object_iter_value(iter);
+    if (json_typeof(value) == JSON_TRUE)  {
+        result = 0;
+    } else {
+        result = EIO;  // FALSE means error in remote NN/DN
     }
     }
-    return fileStat;
+    json_decref(root);
+    return result;
 }
 }
 
 
+int parseMKDIR(const char *response)
+{
+    return parseBoolean(response);
+}
+
+int parseRENAME(const char *response)
+{
+    return parseBoolean(response);
+}
+
+int parseDELETE(const char *response)
+{
+    return parseBoolean(response);
+}
 
 
-int checkHeader(char *header, const char *content, const char *operation) {
+int parseSETREPLICATION(const char *response)
+{
+    return parseBoolean(response);
+}
+
+/**
+ * Check the header of response to see if it's 200 OK
+ * 
+ * @param header    Header information for checking
+ * @param content   Stores exception information if there are errors
+ * @param operation Indicate the operation for exception printing
+ * @return 0 for success
+ */
+static int checkHeader(const char *header, const char *content,
+                       const char *operation)
+{
     char *result = NULL;
     char *result = NULL;
-    char delims[] = ":";
-    char *responseCode= "200 OK";
-    if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) {
-        return 0;
+    const char delims[] = ":";
+    char *savepter;
+    int ret = 0;
+    
+    if (!header || strncmp(header, "HTTP/", strlen("HTTP/"))) {
+        return EINVAL;
     }
     }
-    if(!(strstr(header, responseCode)) || !(header = strstr(header, "Content-Length"))) {
+    if (!(strstr(header, twoHundredOKCode)) ||
+       !(result = strstr(header, "Content-Length"))) {
         struct jsonException *exc = parseException(content);
         struct jsonException *exc = parseException(content);
         if (exc) {
         if (exc) {
-            errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
+            ret = printJsonException(exc, PRINT_EXC_ALL,
+                                       "Calling WEBHDFS (%s)", operation);
+        } else {
+            ret = EIO;
         }
         }
-        return 0;
+        return ret;
     }
     }
-    result = strtok(header, delims);
-    result = strtok(NULL,delims);
+    result = strtok_r(result, delims, &savepter);
+    result = strtok_r(NULL, delims, &savepter);
     while (isspace(*result)) {
     while (isspace(*result)) {
         result++;
         result++;
     }
     }
-    if(strcmp(result,"0")) {                 //Content-Length should be equal to 0
-        return 1;
-    } else {
-        return 0;
+    // Content-Length should be equal to 0,
+    // and the string should be "0\r\nServer"
+    if (strncmp(result, "0\r\n", 3)) {
+        ret = EIO;
     }
     }
+    return ret;
 }
 }
 
 
-int parseOPEN(const char *header, const char *content) {
-    const char *responseCode1 = "307 TEMPORARY_REDIRECT";
-    const char *responseCode2 = "200 OK";
-    if(header == '\0' || strncmp(header,"HTTP/",strlen("HTTP/"))) {
-        return -1;
-    }
-    if(!(strstr(header,responseCode1) && strstr(header, responseCode2))) {
-        struct jsonException *exc = parseException(content);
-        if (exc) {
-            //if the exception is an IOException and it is because the offset is out of the range
-            //do not print out the exception
-            if (!strcasecmp(exc->exception, "IOException") && strstr(exc->message, "out of the range")) {
-                return 0;
-            }
-            errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (OPEN)");
-        }
-        return -1;
-    }
-    
-    return 1;
-}
-
-int parseCHMOD(char *header, const char *content) {
+int parseCHMOD(const char *header, const char *content)
+{
     return checkHeader(header, content, "CHMOD");
     return checkHeader(header, content, "CHMOD");
 }
 }
 
 
-
-int parseCHOWN(char *header, const char *content) {
+int parseCHOWN(const char *header, const char *content)
+{
     return checkHeader(header, content, "CHOWN");
     return checkHeader(header, content, "CHOWN");
 }
 }
 
 
-int parseUTIMES(char *header, const char *content) {
-    return checkHeader(header, content, "UTIMES");
+int parseUTIMES(const char *header, const char *content)
+{
+    return checkHeader(header, content, "SETTIMES");
 }
 }
 
 
-
-int checkIfRedirect(const char *const headerstr, const char *content, const char *operation) {
-    char *responseCode = "307 TEMPORARY_REDIRECT";
-    char * locTag = "Location";
-    char * tempHeader;
-    if(headerstr == '\0' || strncmp(headerstr,"HTTP/", 5)) {
-        return 0;
-    }
-    if(!(tempHeader = strstr(headerstr,responseCode))) {
-        //process possible exception information
+/**
+ * Check if the header contains correct information
+ * ("307 TEMPORARY_REDIRECT" and "Location")
+ * 
+ * @param header    Header for parsing
+ * @param content   Contains exception information 
+ *                  if the remote operation failed
+ * @param operation Specify the remote operation when printing out exception
+ * @return 0 for success
+ */
+static int checkRedirect(const char *header,
+                         const char *content, const char *operation)
+{
+    const char *locTag = "Location";
+    int ret = 0, offset = 0;
+    
+    // The header must start with "HTTP/1.1"
+    if (!header || strncmp(header, httpHeaderString,
+                           strlen(httpHeaderString))) {
+        return EINVAL;
+    }
+    
+    offset += strlen(httpHeaderString);
+    while (isspace(header[offset])) {
+        offset++;
+    }
+    // Looking for "307 TEMPORARY_REDIRECT" in header
+    if (strncmp(header + offset, temporaryRedirectCode,
+                strlen(temporaryRedirectCode))) {
+        // Process possible exception information
         struct jsonException *exc = parseException(content);
         struct jsonException *exc = parseException(content);
         if (exc) {
         if (exc) {
-            errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation);
+            ret = printJsonException(exc, PRINT_EXC_ALL,
+                                     "Calling WEBHDFS (%s)", operation);
+        } else {
+            ret = EIO;
         }
         }
-        return 0;
+        return ret;
     }
     }
-    if(!(strstr(tempHeader,locTag))) {
-        return 0;
+    // Here we just simply check if header contains "Location" tag,
+    // detailed processing is in parseDnLoc
+    if (!(strstr(header, locTag))) {
+        ret = EIO;
     }
     }
-    return 1;
+    return ret;
 }
 }
 
 
-
-int parseNnWRITE(const char *header, const char *content) {
-    return checkIfRedirect(header, content, "Write(NameNode)");
+int parseNnWRITE(const char *header, const char *content)
+{
+    return checkRedirect(header, content, "Write(NameNode)");
 }
 }
 
 
+int parseNnAPPEND(const char *header, const char *content)
+{
+    return checkRedirect(header, content, "Append(NameNode)");
+}
 
 
-int parseNnAPPEND(const char *header, const char *content) {
-    return checkIfRedirect(header, content, "Append(NameNode)");
+/** 0 for success , -1 for out of range, other values for error */
+int parseOPEN(const char *header, const char *content)
+{
+    int ret = 0, offset = 0;
+    
+    if (!header || strncmp(header, httpHeaderString,
+                           strlen(httpHeaderString))) {
+        return EINVAL;
+    }
+    
+    offset += strlen(httpHeaderString);
+    while (isspace(header[offset])) {
+        offset++;
+    }
+    if (strncmp(header + offset, temporaryRedirectCode,
+                strlen(temporaryRedirectCode)) ||
+        !strstr(header, twoHundredOKCode)) {
+        struct jsonException *exc = parseException(content);
+        if (exc) {
+            // If the exception is an IOException and it is because
+            // the offset is out of the range, do not print out the exception
+            if (!strcasecmp(exc->exception, "IOException") &&
+                    strstr(exc->message, "out of the range")) {
+                ret = -1;
+            } else {
+                ret = printJsonException(exc, PRINT_EXC_ALL,
+                                       "Calling WEBHDFS (OPEN)");
+            }
+        } else {
+            ret = EIO;
+        }
+    }
+    return ret;
 }
 }
 
 
-char *parseDnLoc(char *content) {
-    char delims[] = "\r\n";
-    char *url = NULL;
-    char *DnLocation = NULL;
-    char *savepter;
-    DnLocation = strtok_r(content, delims, &savepter);
-    while (DnLocation && strncmp(DnLocation, "Location:", strlen("Location:"))) {
-        DnLocation = strtok_r(NULL, delims, &savepter);
+int parseDnLoc(char *content, char **dn)
+{
+    char *url = NULL, *dnLocation = NULL, *savepter, *tempContent;
+    const char *prefix = "Location: http://";
+    const char *prefixToRemove = "Location: ";
+    const char *delims = "\r\n";
+    
+    tempContent = strdup(content);
+    if (!tempContent) {
+        return ENOMEM;
     }
     }
-    if (!DnLocation) {
-        return NULL;
+    
+    dnLocation = strtok_r(tempContent, delims, &savepter);
+    while (dnLocation && strncmp(dnLocation, "Location:",
+                                 strlen("Location:"))) {
+        dnLocation = strtok_r(NULL, delims, &savepter);
     }
     }
-    DnLocation = strstr(DnLocation, "http");
-    if (!DnLocation) {
-        return NULL;
+    if (!dnLocation) {
+        return EIO;
     }
     }
-    url = malloc(strlen(DnLocation) + 1);
+    
+    while (isspace(*dnLocation)) {
+        dnLocation++;
+    }
+    if (strncmp(dnLocation, prefix, strlen(prefix))) {
+        return EIO;
+    }
+    url = strdup(dnLocation + strlen(prefixToRemove));
     if (!url) {
     if (!url) {
-        return NULL;
+        return ENOMEM;
     }
     }
-    strcpy(url, DnLocation);
-    return url;
+    *dn = url;
+    return 0;
 }
 }
 
 
-int parseDnWRITE(const char *header, const char *content) {
-    char *responseCode = "201 Created";
-    fprintf(stderr, "\nheaderstr is: %s\n", header);
-    if(header == '\0' || strncmp(header,"HTTP/",strlen("HTTP/"))) {
-        return 0;
+int parseDnWRITE(const char *header, const char *content)
+{
+    int ret = 0;
+    if (header == NULL || header[0] == '\0' ||
+                         strncmp(header, "HTTP/", strlen("HTTP/"))) {
+        return EINVAL;
     }
     }
-    if(!(strstr(header,responseCode))) {
+    if (!(strstr(header, twoHundredOneCreatedCode))) {
         struct jsonException *exc = parseException(content);
         struct jsonException *exc = parseException(content);
         if (exc) {
         if (exc) {
-            errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (WRITE(DataNode))");
+            ret = printJsonException(exc, PRINT_EXC_ALL,
+                                     "Calling WEBHDFS (WRITE(DataNode))");
+        } else {
+            ret = EIO;
         }
         }
-        return 0;
     }
     }
-    return 1;
+    return ret;
 }
 }
 
 
-int parseDnAPPEND(const char *header, const char *content) {
-    char *responseCode = "200 OK";
-    if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) {
-        return 0;
+int parseDnAPPEND(const char *header, const char *content)
+{
+    int ret = 0;
+    
+    if (header == NULL || header[0] == '\0' ||
+                         strncmp(header, "HTTP/", strlen("HTTP/"))) {
+        return EINVAL;
     }
     }
-    if(!(strstr(header, responseCode))) {
+    if (!(strstr(header, twoHundredOKCode))) {
         struct jsonException *exc = parseException(content);
         struct jsonException *exc = parseException(content);
         if (exc) {
         if (exc) {
-            errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (APPEND(DataNode))");
+            ret = printJsonException(exc, PRINT_EXC_ALL,
+                                     "Calling WEBHDFS (APPEND(DataNode))");
+        } else {
+            ret = EIO;
+        }
+    }
+    return ret;
+}
+
+/**
+ * Retrieve file status from the JSON object 
+ *
+ * @param jobj          JSON object for parsing, which contains 
+ *                      file status information
+ * @param fileStat      hdfsFileInfo handle to hold file status information
+ * @return 0 on success
+ */
+static int parseJsonForFileStatus(json_t *jobj, hdfsFileInfo *fileStat)
+{
+    const char *key, *tempstr;
+    json_t *value;
+    void *iter = NULL;
+    
+    iter = json_object_iter(jobj);
+    while (iter) {
+        key = json_object_iter_key(iter);
+        value = json_object_iter_value(iter);
+        
+        if (!strcmp(key, "accessTime")) {
+            // json field contains time in milliseconds,
+            // hdfsFileInfo is counted in seconds
+            fileStat->mLastAccess = json_integer_value(value) / 1000;
+        } else if (!strcmp(key, "blockSize")) {
+            fileStat->mBlockSize = json_integer_value(value);
+        } else if (!strcmp(key, "length")) {
+            fileStat->mSize = json_integer_value(value);
+        } else if (!strcmp(key, "modificationTime")) {
+            fileStat->mLastMod = json_integer_value(value) / 1000;
+        } else if (!strcmp(key, "replication")) {
+            fileStat->mReplication = json_integer_value(value);
+        } else if (!strcmp(key, "group")) {
+            fileStat->mGroup = strdup(json_string_value(value));
+            if (!fileStat->mGroup) {
+                return ENOMEM;
+            }
+        } else if (!strcmp(key, "owner")) {
+            fileStat->mOwner = strdup(json_string_value(value));
+            if (!fileStat->mOwner) {
+                return ENOMEM;
+            }
+        } else if (!strcmp(key, "pathSuffix")) {
+            fileStat->mName = strdup(json_string_value(value));
+            if (!fileStat->mName) {
+                return ENOMEM;
+            }
+        } else if (!strcmp(key, "permission")) {
+            tempstr = json_string_value(value);
+            fileStat->mPermissions = (short) strtol(tempstr, NULL, 8);
+        } else if (!strcmp(key, "type")) {
+            tempstr = json_string_value(value);
+            if (!strcmp(tempstr, "DIRECTORY")) {
+                fileStat->mKind = kObjectKindDirectory;
+            } else {
+                fileStat->mKind = kObjectKindFile;
+            }
         }
         }
-        return 0;
+        // Go to the next key-value pair in the json object
+        iter = json_object_iter_next(jobj, iter);
     }
     }
-    return 1;
+    return 0;
 }
 }
 
 
-hdfsFileInfo *parseGFS(char *str, hdfsFileInfo *fileStat, int *numEntries) {
+int parseGFS(const char *response, hdfsFileInfo *fileStat, int printError)
+{
+    int ret = 0, printFlag;
     json_error_t error;
     json_error_t error;
     size_t flags = 0;
     size_t flags = 0;
-    json_t *jobj = json_loads(str, flags, &error);
-    fileStat = parseJsonGFS(jobj, fileStat, numEntries, "GETPATHSTATUS/LISTSTATUS");
-    return fileStat;
+    json_t *jobj, *value;
+    const char *key;
+    void *iter = NULL;
+    
+    if (!response || !fileStat) {
+        return EIO;
+    }
+    jobj = json_loads(response, flags, &error);
+    if (!jobj) {
+        fprintf(stderr, "error while parsing json: on line %d: %s\n",
+                error.line, error.text);
+        return EIO;
+    }
+    iter = json_object_iter(jobj);
+    key = json_object_iter_key(iter);
+    value = json_object_iter_value(iter);
+    if (json_typeof(value) == JSON_OBJECT) {
+        if (!strcmp(key, "RemoteException")) {
+            struct jsonException *exception = parseJsonException(value);
+            if (exception) {
+                if (printError) {
+                    printFlag = PRINT_EXC_ALL;
+                } else {
+                    printFlag = NOPRINT_EXC_FILE_NOT_FOUND |
+                                NOPRINT_EXC_ACCESS_CONTROL |
+                                NOPRINT_EXC_PARENT_NOT_DIRECTORY;
+                }
+                ret = printJsonException(exception, printFlag,
+                                         "Calling WEBHDFS GETFILESTATUS");
+            } else {
+                ret = EIO;
+            }
+        } else if (!strcmp(key, "FileStatus")) {
+            ret = parseJsonForFileStatus(value, fileStat);
+        } else {
+            ret = EIO;
+        }
+        
+    } else {
+        ret = EIO;
+    }
+    
+    json_decref(jobj);
+    return ret;
 }
 }
 
 
-int parseSETREPLICATION(char *response) {
-    return (parseBoolean(response));
+/**
+ * Parse the JSON array. Called to parse the result of 
+ * the LISTSTATUS operation. Thus each element of the JSON array is 
+ * a JSON object with the information of a file entry contained 
+ * in the folder.
+ *
+ * @param jobj          The JSON array to be parsed
+ * @param fileStat      The hdfsFileInfo handle used to 
+ *                      store a group of file information
+ * @param numEntries    Capture the number of files in the folder
+ * @return              0 for success
+ */
+static int parseJsonArrayForFileStatuses(json_t *jobj, hdfsFileInfo **fileStat,
+                                         int *numEntries)
+{
+    json_t *jvalue = NULL;
+    int i = 0, ret = 0, arraylen = 0;
+    hdfsFileInfo *fileInfo = NULL;
+    
+    arraylen = (int) json_array_size(jobj);
+    if (arraylen > 0) {
+        fileInfo = calloc(arraylen, sizeof(hdfsFileInfo));
+        if (!fileInfo) {
+            return ENOMEM;
+        }
+    }
+    for (i = 0; i < arraylen; i++) {
+        //Getting the array element at position i
+        jvalue = json_array_get(jobj, i);
+        if (json_is_object(jvalue)) {
+            ret = parseJsonForFileStatus(jvalue, &fileInfo[i]);
+            if (ret) {
+                goto done;
+            }
+        } else {
+            ret = EIO;
+            goto done;
+        }
+    }
+done:
+    if (ret) {
+        free(fileInfo);
+    } else {
+        *numEntries = arraylen;
+        *fileStat = fileInfo;
+    }
+    return ret;
 }
 }
 
 
+int parseLS(const char *response, hdfsFileInfo **fileStats, int *numOfEntries)
+{
+    int ret = 0;
+    json_error_t error;
+    size_t flags = 0;
+    json_t *jobj, *value;
+    const char *key;
+    void *iter = NULL;
+    
+    if (!response || response[0] == '\0' || !fileStats) {
+        return EIO;
+    }
+    jobj = json_loads(response, flags, &error);
+    if (!jobj) {
+        fprintf(stderr, "error while parsing json: on line %d: %s\n",
+                error.line, error.text);
+        return EIO;
+    }
+    
+    iter = json_object_iter(jobj);
+    key = json_object_iter_key(iter);
+    value = json_object_iter_value(iter);
+    if (json_typeof(value) == JSON_OBJECT) {
+        if (!strcmp(key, "RemoteException")) {
+            struct jsonException *exception = parseJsonException(value);
+            if (exception) {
+                ret = printJsonException(exception, PRINT_EXC_ALL,
+                                         "Calling WEBHDFS GETFILESTATUS");
+            } else {
+                ret = EIO;
+            }
+        } else if (!strcmp(key, "FileStatuses")) {
+            iter = json_object_iter(value);
+            value = json_object_iter_value(iter);
+            if (json_is_array(value)) {
+                ret = parseJsonArrayForFileStatuses(value, fileStats,
+                                                    numOfEntries);
+            } else {
+                ret = EIO;
+            }
+        } else {
+            ret = EIO;
+        }
+    } else {
+        ret = EIO;
+    }
+    
+    json_decref(jobj);
+    return ret;
+}

+ 141 - 21
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_json_parser.h

@@ -18,41 +18,161 @@
 #ifndef _HDFS_JSON_PARSER_H_
 #ifndef _HDFS_JSON_PARSER_H_
 #define _HDFS_JSON_PARSER_H_
 #define _HDFS_JSON_PARSER_H_
 
 
-struct jsonException;
+/**
+ * Parse the response for MKDIR request. The response uses TRUE/FALSE 
+ * to indicate whether the operation succeeded.
+ *
+ * @param response  The response information to parse.
+ * @return 0 for success
+ */
+int parseMKDIR(const char *response);
 
 
 /**
 /**
- * Print out JSON exception information.
+ * Parse the response for RENAME request. The response uses TRUE/FALSE
+ * to indicate whether the operation succeeded.
  *
  *
- * @param exc             The exception information to print and free
- * @param noPrintFlags    Flags which determine which exceptions we should NOT
- *                        print.
- * @param fmt             Printf-style format list
- * @param ...             Printf-style varargs
+ * @param response  The response information to parse.
+ * @return 0 for success
+ */
+int parseRENAME(const char *response);
+
+/**
+ * Parse the response for DELETE request. The response uses TRUE/FALSE
+ * to indicate whether the operation succeeded.
  *
  *
- * @return                The POSIX error number associated with the exception
- *                        object.
+ * @param response  The response information to parse.
+ * @return 0 for success
  */
  */
-int printJsonException(struct jsonException *exc, int noPrintFlags,
-                       const char *fmt, ...);
+int parseDELETE(const char *response);
 
 
-int parseMKDIR(char *response);
-int parseRENAME(char *response);
-int parseDELETE (char *response);
-int parseSETREPLICATION(char *response);
+/**
+ * Parse the response for SETREPLICATION request. The response uses TRUE/FALSE
+ * to indicate whether the operation succeeded.
+ *
+ * @param response  The response information to parse.
+ * @return 0 for success
+ */
+int parseSETREPLICATION(const char *response);
 
 
+/**
+ * Parse the response for OPEN (read) request. A successful operation 
+ * will return "200 OK".
+ *
+ * @param response  The response information for parsing
+ * @return          0 for success , -1 for out of range, other values for error
+ */
 int parseOPEN(const char *header, const char *content);
 int parseOPEN(const char *header, const char *content);
 
 
+/**
+ * Parse the response for WRITE (from NameNode) request. 
+ * A successful operation should return "307 TEMPORARY_REDIRECT" in its header.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message 
+ *                  sent from NameNode is stored in content
+ * @return          0 for success
+ */
 int parseNnWRITE(const char *header, const char *content);
 int parseNnWRITE(const char *header, const char *content);
+
+/**
+ * Parse the response for WRITE (from DataNode) request. 
+ * A successful operation should return "201 Created" in its header.
+ * 
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message
+ *                  sent from DataNode is stored in content
+ * @return          0 for success
+ */
 int parseDnWRITE(const char *header, const char *content);
 int parseDnWRITE(const char *header, const char *content);
+
+/**
+ * Parse the response for APPEND (sent from NameNode) request.
+ * A successful operation should return "307 TEMPORARY_REDIRECT" in its header.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message
+ *                  sent from NameNode is stored in content
+ * @return          0 for success
+ */
 int parseNnAPPEND(const char *header, const char *content);
 int parseNnAPPEND(const char *header, const char *content);
+
+/**
+ * Parse the response for APPEND (from DataNode) request.
+ * A successful operation should return "200 OK" in its header.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message
+ *                  sent from DataNode is stored in content
+ * @return          0 for success
+ */
 int parseDnAPPEND(const char *header, const char *content);
 int parseDnAPPEND(const char *header, const char *content);
 
 
-char* parseDnLoc(char *content);
+/**
+ * Parse the response (from NameNode) to get the location information 
+ * of the DataNode that should be contacted for the following write operation.
+ *
+ * @param content   Content of the http header
+ * @param dn        To store the location of the DataNode for writing
+ * @return          0 for success
+ */
+int parseDnLoc(char *content, char **dn) __attribute__ ((warn_unused_result));
+
+/**
+ * Parse the response for GETFILESTATUS operation.
+ *
+ * @param response      Response to parse. Its detailed format is specified in 
+ *            "http://hadoop.apache.org/docs/stable/webhdfs.html#GETFILESTATUS"
+ * @param fileStat      A hdfsFileInfo handle for holding file information
+ * @param printError    Whether or not print out exception 
+ *                      when file does not exist
+ * @return 0 for success, non-zero value to indicate error
+ */
+int parseGFS(const char *response, hdfsFileInfo *fileStat, int printError);
 
 
-hdfsFileInfo *parseGFS(char *response, hdfsFileInfo *fileStat, int *numEntries);
+/**
+ * Parse the response for LISTSTATUS operation.
+ *
+ * @param response      Response to parse. Its detailed format is specified in
+ *            "http://hadoop.apache.org/docs/r1.0.3/webhdfs.html#LISTSTATUS"
+ * @param fileStats     Pointer pointing to a list of hdfsFileInfo handles 
+ *                      holding file/dir information in the directory
+ * @param numEntries    After parsing, the value of this parameter indicates
+ *                      the number of file entries.
+ * @return 0 for success, non-zero value to indicate error
+ */
+int parseLS(const char *response, hdfsFileInfo **fileStats, int *numOfEntries);
 
 
-int parseCHOWN (char *header, const char *content);
-int parseCHMOD (char *header, const char *content);
-int parseUTIMES(char *header, const char *content);
+/**
+ * Parse the response for CHOWN request.
+ * A successful operation should contains "200 OK" in its header, 
+ * and the Content-Length should be 0.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message is stored in content
+ * @return          0 for success
+ */
+int parseCHOWN(const char *header, const char *content);
+
+/**
+ * Parse the response for CHMOD request.
+ * A successful operation should contains "200 OK" in its header,
+ * and the Content-Length should be 0.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message is stored in content
+ * @return          0 for success
+ */
+int parseCHMOD(const char *header, const char *content);
+
+/**
+ * Parse the response for SETTIMES request.
+ * A successful operation should contains "200 OK" in its header,
+ * and the Content-Length should be 0.
+ *
+ * @param header    The header of the http response
+ * @param content   If failing, the exception message is stored in content
+ * @return          0 for success
+ */
+int parseUTIMES(const char *header, const char *content);
 
 
-#endif //_FUSE_JSON_PARSER_H
+#endif //_HDFS_JSON_PARSER_H_

文件差異過大導致無法顯示
+ 379 - 230
hadoop-hdfs-project/hadoop-hdfs/src/contrib/libwebhdfs/src/hdfs_web.c


+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -651,7 +651,7 @@ public class DFSClient implements java.io.Closeable {
       // if there is no more clients under the renewer.
       // if there is no more clients under the renewer.
       getLeaseRenewer().closeClient(this);
       getLeaseRenewer().closeClient(this);
     } catch (IOException ioe) {
     } catch (IOException ioe) {
-       LOG.info("Exception occurred while aborting the client. " + ioe);
+       LOG.info("Exception occurred while aborting the client " + ioe);
     }
     }
     closeConnectionToNamenode();
     closeConnectionToNamenode();
   }
   }
@@ -2139,7 +2139,7 @@ public class DFSClient implements java.io.Closeable {
       reportBadBlocks(lblocks);
       reportBadBlocks(lblocks);
     } catch (IOException ie) {
     } catch (IOException ie) {
       LOG.info("Found corruption while reading " + file
       LOG.info("Found corruption while reading " + file
-          + ".  Error repairing corrupt blocks.  Bad blocks remain.", ie);
+          + ". Error repairing corrupt blocks. Bad blocks remain.", ie);
     }
     }
   }
   }
 
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -457,7 +457,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
             buffersize, verifyChecksum, dfsClient.clientName);
             buffersize, verifyChecksum, dfsClient.clientName);
         if(connectFailedOnce) {
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
-                             " for block " + blk.getBlockId());
+                             " for " + blk);
         }
         }
         return chosenNode;
         return chosenNode;
       } catch (IOException ex) {
       } catch (IOException ex) {
@@ -736,9 +736,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         }
         }
         
         
         if (nodes == null || nodes.length == 0) {
         if (nodes == null || nodes.length == 0) {
-          DFSClient.LOG.info("No node available for block: " + blockInfo);
+          DFSClient.LOG.info("No node available for " + blockInfo);
         }
         }
-        DFSClient.LOG.info("Could not obtain block " + block.getBlock()
+        DFSClient.LOG.info("Could not obtain " + block.getBlock()
             + " from any node: " + ie
             + " from any node: " + ie
             + ". Will get new block locations from namenode and retry...");
             + ". Will get new block locations from namenode and retry...");
         try {
         try {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -735,7 +735,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
     //
     //
     private boolean processDatanodeError() throws IOException {
     private boolean processDatanodeError() throws IOException {
       if (response != null) {
       if (response != null) {
-        DFSClient.LOG.info("Error Recovery for block " + block +
+        DFSClient.LOG.info("Error Recovery for " + block +
         " waiting for responder to exit. ");
         " waiting for responder to exit. ");
         return true;
         return true;
       }
       }
@@ -1008,7 +1008,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         success = createBlockOutputStream(nodes, 0L, false);
         success = createBlockOutputStream(nodes, 0L, false);
 
 
         if (!success) {
         if (!success) {
-          DFSClient.LOG.info("Abandoning block " + block);
+          DFSClient.LOG.info("Abandoning " + block);
           dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName);
           dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName);
           block = null;
           block = null;
           DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
           DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
@@ -1773,7 +1773,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         try {
         try {
           Thread.sleep(400);
           Thread.sleep(400);
           if (Time.now() - localstart > 5000) {
           if (Time.now() - localstart > 5000) {
-            DFSClient.LOG.info("Could not complete file " + src + " retrying...");
+            DFSClient.LOG.info("Could not complete " + src + " retrying...");
           }
           }
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
           DFSClient.LOG.warn("Caught exception ", ie);
           DFSClient.LOG.warn("Caught exception ", ie);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -702,7 +702,7 @@ public class DistributedFileSystem extends FileSystem {
     }
     }
     DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
     DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
     lblocks[0] = new LocatedBlock(dataBlock, dataNode);
     lblocks[0] = new LocatedBlock(dataBlock, dataNode);
-    LOG.info("Found checksum error in data stream at block="
+    LOG.info("Found checksum error in data stream at "
         + dataBlock + " on datanode="
         + dataBlock + " on datanode="
         + dataNode[0]);
         + dataNode[0]);
 
 
@@ -715,7 +715,7 @@ public class DistributedFileSystem extends FileSystem {
     }
     }
     DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
     DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
     lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
     lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
-    LOG.info("Found checksum error in checksum stream at block="
+    LOG.info("Found checksum error in checksum stream at "
         + sumsBlock + " on datanode=" + sumsNode[0]);
         + sumsBlock + " on datanode=" + sumsNode[0]);
 
 
     // Ask client to delete blocks.
     // Ask client to delete blocks.

+ 27 - 33
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -950,8 +950,8 @@ public class BlockManager {
       datanodes.append(node).append(" ");
       datanodes.append(node).append(" ");
     }
     }
     if (datanodes.length() != 0) {
     if (datanodes.length() != 0) {
-      NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
-          + b + " to " + datanodes.toString());
+      NameNode.stateChangeLog.info("BLOCK* addToInvalidates: " + b + " "
+          + datanodes);
     }
     }
   }
   }
 
 
@@ -972,7 +972,7 @@ public class BlockManager {
       // thread of Datanode reports bad block before Block reports are sent
       // thread of Datanode reports bad block before Block reports are sent
       // by the Datanode on startup
       // by the Datanode on startup
       NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
       NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
-          + blk + " not found.");
+          + blk + " not found");
       return;
       return;
     }
     }
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn);
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn);
@@ -1026,7 +1026,7 @@ public class BlockManager {
       NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " +
       NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " +
           "invalidation of " + b + " on " + dn + " because " +
           "invalidation of " + b + " on " + dn + " because " +
           nr.replicasOnStaleNodes() + " replica(s) are located on nodes " +
           nr.replicasOnStaleNodes() + " replica(s) are located on nodes " +
-          "with potentially out-of-date block reports.");
+          "with potentially out-of-date block reports");
       postponeBlock(b.corrupted);
       postponeBlock(b.corrupted);
 
 
     } else if (nr.liveReplicas() >= 1) {
     } else if (nr.liveReplicas() >= 1) {
@@ -1039,7 +1039,7 @@ public class BlockManager {
       }
       }
     } else {
     } else {
       NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + b
       NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + b
-          + " on " + dn + " is the only copy and was not deleted.");
+          + " on " + dn + " is the only copy and was not deleted");
     }
     }
   }
   }
 
 
@@ -1160,9 +1160,8 @@ public class BlockManager {
                    (blockHasEnoughRacks(block)) ) {
                    (blockHasEnoughRacks(block)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
                 neededReplications.remove(block, priority); // remove from neededReplications
                 neededReplications.decrementReplicationIndex(priority);
                 neededReplications.decrementReplicationIndex(priority);
-                NameNode.stateChangeLog.info("BLOCK* "
-                    + "Removing block " + block
-                    + " from neededReplications as it has enough replicas.");
+                NameNode.stateChangeLog.info("BLOCK* Removing " + block
+                    + " from neededReplications as it has enough replicas");
                 continue;
                 continue;
               }
               }
             }
             }
@@ -1236,9 +1235,8 @@ public class BlockManager {
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.decrementReplicationIndex(priority);
               neededReplications.decrementReplicationIndex(priority);
               rw.targets = null;
               rw.targets = null;
-              NameNode.stateChangeLog.info("BLOCK* "
-                  + "Removing block " + block
-                  + " from neededReplications as it has enough replicas.");
+              NameNode.stateChangeLog.info("BLOCK* Removing " + block
+                  + " from neededReplications as it has enough replicas");
               continue;
               continue;
             }
             }
           }
           }
@@ -1290,10 +1288,8 @@ public class BlockManager {
             targetList.append(' ');
             targetList.append(' ');
             targetList.append(targets[k]);
             targetList.append(targets[k]);
           }
           }
-          NameNode.stateChangeLog.info(
-                  "BLOCK* ask "
-                  + rw.srcNode + " to replicate "
-                  + rw.block + " to " + targetList);
+          NameNode.stateChangeLog.info("BLOCK* ask " + rw.srcNode
+              + " to replicate " + rw.block + " to " + targetList);
         }
         }
       }
       }
     }
     }
@@ -1527,10 +1523,9 @@ public class BlockManager {
       boolean staleBefore = node.areBlockContentsStale();
       boolean staleBefore = node.areBlockContentsStale();
       node.receivedBlockReport();
       node.receivedBlockReport();
       if (staleBefore && !node.areBlockContentsStale()) {
       if (staleBefore && !node.areBlockContentsStale()) {
-        LOG.info("BLOCK* processReport: " +
-            "Received first block report from " + node +
-            " after becoming active. Its block contents are no longer" +
-            " considered stale.");
+        LOG.info("BLOCK* processReport: Received first block report from "
+            + node + " after becoming active. Its block contents are no longer"
+            + " considered stale");
         rescanPostponedMisreplicatedBlocks();
         rescanPostponedMisreplicatedBlocks();
       }
       }
       
       
@@ -1601,9 +1596,9 @@ public class BlockManager {
       addStoredBlock(b, node, null, true);
       addStoredBlock(b, node, null, true);
     }
     }
     for (Block b : toInvalidate) {
     for (Block b : toInvalidate) {
-      NameNode.stateChangeLog.info("BLOCK* processReport: block "
+      NameNode.stateChangeLog.info("BLOCK* processReport: "
           + b + " on " + node + " size " + b.getNumBytes()
           + b + " on " + node + " size " + b.getNumBytes()
-          + " does not belong to any file.");
+          + " does not belong to any file");
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
     for (BlockToMarkCorrupt b : toCorrupt) {
@@ -1870,7 +1865,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int count = pendingDNMessages.count();
     int count = pendingDNMessages.count();
     if (count > 0) {
     if (count > 0) {
       LOG.info("Processing " + count + " messages from DataNodes " +
       LOG.info("Processing " + count + " messages from DataNodes " +
-          "that were previously queued during standby state.");
+          "that were previously queued during standby state");
     }
     }
     processQueuedMessages(pendingDNMessages.takeAll());
     processQueuedMessages(pendingDNMessages.takeAll());
     assert pendingDNMessages.count() == 0;
     assert pendingDNMessages.count() == 0;
@@ -1927,9 +1922,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
           // the block report got a little bit delayed after the pipeline
           // the block report got a little bit delayed after the pipeline
           // closed. So, ignore this report, assuming we will get a
           // closed. So, ignore this report, assuming we will get a
           // FINALIZED replica later. See HDFS-2791
           // FINALIZED replica later. See HDFS-2791
-          LOG.info("Received an RBW replica for block " + storedBlock +
-              " on " + dn + ": ignoring it, since the block is " +
-              "complete with the same generation stamp.");
+          LOG.info("Received an RBW replica for " + storedBlock +
+              " on " + dn + ": ignoring it, since it is " +
+              "complete with the same genstamp");
           return null;
           return null;
         } else {
         } else {
           return new BlockToMarkCorrupt(storedBlock,
           return new BlockToMarkCorrupt(storedBlock,
@@ -2041,7 +2036,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       // If this block does not belong to anyfile, then we are done.
       // If this block does not belong to anyfile, then we are done.
       NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
       NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
           + node + " size " + block.getNumBytes()
           + node + " size " + block.getNumBytes()
-          + " but it does not belong to any file.");
+          + " but it does not belong to any file");
       // we could add this block to invalidate set of this datanode.
       // we could add this block to invalidate set of this datanode.
       // it will happen in next block report otherwise.
       // it will happen in next block report otherwise.
       return block;
       return block;
@@ -2158,9 +2153,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       try {
       try {
         invalidateBlock(new BlockToMarkCorrupt(blk, null), node);
         invalidateBlock(new BlockToMarkCorrupt(blk, null), node);
       } catch (IOException e) {
       } catch (IOException e) {
-        NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " +
-                                      "error in deleting bad block " + blk +
-                                      " on " + node, e);
+        NameNode.stateChangeLog.info("invalidateCorruptReplicas "
+            + "error in deleting bad block " + blk + " on " + node, e);
         gotException = true;
         gotException = true;
       }
       }
     }
     }
@@ -2308,7 +2302,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       DatanodeDescriptor cur = it.next();
       DatanodeDescriptor cur = it.next();
       if (cur.areBlockContentsStale()) {
       if (cur.areBlockContentsStale()) {
         LOG.info("BLOCK* processOverReplicatedBlock: " +
         LOG.info("BLOCK* processOverReplicatedBlock: " +
-            "Postponing processing of over-replicated block " +
+            "Postponing processing of over-replicated " +
             block + " since datanode " + cur + " does not yet have up-to-date " +
             block + " since datanode " + cur + " does not yet have up-to-date " +
             "block information.");
             "block information.");
         postponeBlock(block);
         postponeBlock(block);
@@ -2398,7 +2392,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       //
       //
       addToInvalidates(b, cur);
       addToInvalidates(b, cur);
       NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
       NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
-                +"("+cur+", "+b+") is added to invalidated blocks set.");
+                +"("+cur+", "+b+") is added to invalidated blocks set");
     }
     }
   }
   }
 
 
@@ -2540,7 +2534,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     for (Block b : toInvalidate) {
     for (Block b : toInvalidate) {
       NameNode.stateChangeLog.info("BLOCK* addBlock: block "
       NameNode.stateChangeLog.info("BLOCK* addBlock: block "
           + b + " on " + node + " size " + b.getNumBytes()
           + b + " on " + node + " size " + b.getNumBytes()
-          + " does not belong to any file.");
+          + " does not belong to any file");
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
     for (BlockToMarkCorrupt b : toCorrupt) {
@@ -2651,7 +2645,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * then it gains speed by ignoring issues of excess replicas or nodes
    * that are decommissioned or in process of becoming decommissioned.
    * that are decommissioned or in process of becoming decommissioned.
-   * If not in startup, then it calls {@link countNodes()} instead.
+   * If not in startup, then it calls {@link #countNodes(Block)} instead.
    * 
    * 
    * @param b - the block being tested
    * @param b - the block being tested
    * @return count of live nodes for this block
    * @return count of live nodes for this block

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -362,8 +362,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
   void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       // this prevents adding the same block twice to the recovery queue
-      BlockManager.LOG.info("Block " + block +
-                            " is already in the recovery queue.");
+      BlockManager.LOG.info(block + " is already in the recovery queue");
       return;
       return;
     }
     }
     recoverBlocks.offer(block);
     recoverBlocks.offer(block);

+ 12 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -584,7 +584,7 @@ public class DatanodeManager {
     if (node.isDecommissionInProgress()) {
     if (node.isDecommissionInProgress()) {
       if (!blockManager.isReplicationInProgress(node)) {
       if (!blockManager.isReplicationInProgress(node)) {
         node.setDecommissioned();
         node.setDecommissioned();
-        LOG.info("Decommission complete for node " + node);
+        LOG.info("Decommission complete for " + node);
       }
       }
     }
     }
     return node.isDecommissioned();
     return node.isDecommissioned();
@@ -593,8 +593,8 @@ public class DatanodeManager {
   /** Start decommissioning the specified datanode. */
   /** Start decommissioning the specified datanode. */
   private void startDecommission(DatanodeDescriptor node) {
   private void startDecommission(DatanodeDescriptor node) {
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
     if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
-      LOG.info("Start Decommissioning node " + node + " with " + 
-          node.numBlocks() +  " blocks.");
+      LOG.info("Start Decommissioning " + node + " with " + 
+          node.numBlocks() +  " blocks");
       heartbeatManager.startDecommission(node);
       heartbeatManager.startDecommission(node);
       node.decommissioningStatus.setStartTime(now());
       node.decommissioningStatus.setStartTime(now());
       
       
@@ -606,7 +606,7 @@ public class DatanodeManager {
   /** Stop decommissioning the specified datanodes. */
   /** Stop decommissioning the specified datanodes. */
   void stopDecommission(DatanodeDescriptor node) {
   void stopDecommission(DatanodeDescriptor node) {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      LOG.info("Stop Decommissioning node " + node);
+      LOG.info("Stop Decommissioning " + node);
       heartbeatManager.stopDecommission(node);
       heartbeatManager.stopDecommission(node);
       blockManager.processOverReplicatedBlocksOnReCommission(node);
       blockManager.processOverReplicatedBlocksOnReCommission(node);
     }
     }
@@ -658,17 +658,15 @@ public class DatanodeManager {
       throw new DisallowedDatanodeException(nodeReg);
       throw new DisallowedDatanodeException(nodeReg);
     }
     }
       
       
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.registerDatanode: "
-        + "node registration from " + nodeReg
-        + " storage " + nodeReg.getStorageID());
+    NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
+        + nodeReg + " storage " + nodeReg.getStorageID());
 
 
     DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
     DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
     DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
     DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
         nodeReg.getIpAddr(), nodeReg.getXferPort());
         nodeReg.getIpAddr(), nodeReg.getXferPort());
       
       
     if (nodeN != null && nodeN != nodeS) {
     if (nodeN != null && nodeN != nodeS) {
-      NameNode.LOG.info("BLOCK* NameSystem.registerDatanode: "
-                        + "node from name: " + nodeN);
+      NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN);
       // nodeN previously served a different data storage, 
       // nodeN previously served a different data storage, 
       // which is not served by anybody anymore.
       // which is not served by anybody anymore.
       removeDatanode(nodeN);
       removeDatanode(nodeN);
@@ -683,8 +681,8 @@ public class DatanodeManager {
         // storage. We do not need to remove old data blocks, the delta will
         // storage. We do not need to remove old data blocks, the delta will
         // be calculated on the next block report from the datanode
         // be calculated on the next block report from the datanode
         if(NameNode.stateChangeLog.isDebugEnabled()) {
         if(NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("BLOCK* NameSystem.registerDatanode: "
-                                        + "node restarted.");
+          NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
+              + "node restarted.");
         }
         }
       } else {
       } else {
         // nodeS is found
         // nodeS is found
@@ -696,11 +694,9 @@ public class DatanodeManager {
           value in "VERSION" file under the data directory of the datanode,
           value in "VERSION" file under the data directory of the datanode,
           but this is might not work if VERSION file format has changed 
           but this is might not work if VERSION file format has changed 
        */        
        */        
-        NameNode.stateChangeLog.info( "BLOCK* NameSystem.registerDatanode: "
-                                      + "node " + nodeS
-                                      + " is replaced by " + nodeReg + 
-                                      " with the same storageID " +
-                                      nodeReg.getStorageID());
+        NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
+            + " is replaced by " + nodeReg + " with the same storageID "
+            + nodeReg.getStorageID());
       }
       }
       // update cluster map
       // update cluster map
       getNetworkTopology().remove(nodeS);
       getNetworkTopology().remove(nodeS);

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -433,7 +433,7 @@ public abstract class Storage extends StorageInfo {
         if (!root.exists()) {
         if (!root.exists()) {
           // storage directory does not exist
           // storage directory does not exist
           if (startOpt != StartupOption.FORMAT) {
           if (startOpt != StartupOption.FORMAT) {
-            LOG.info("Storage directory " + rootPath + " does not exist.");
+            LOG.info("Storage directory " + rootPath + " does not exist");
             return StorageState.NON_EXISTENT;
             return StorageState.NON_EXISTENT;
           }
           }
           LOG.info(rootPath + " does not exist. Creating ...");
           LOG.info(rootPath + " does not exist. Creating ...");
@@ -442,7 +442,7 @@ public abstract class Storage extends StorageInfo {
         }
         }
         // or is inaccessible
         // or is inaccessible
         if (!root.isDirectory()) {
         if (!root.isDirectory()) {
-          LOG.info(rootPath + "is not a directory.");
+          LOG.info(rootPath + "is not a directory");
           return StorageState.NON_EXISTENT;
           return StorageState.NON_EXISTENT;
         }
         }
         if (!root.canWrite()) {
         if (!root.canWrite()) {
@@ -539,34 +539,34 @@ public abstract class Storage extends StorageInfo {
       switch(curState) {
       switch(curState) {
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
         LOG.info("Completing previous upgrade for storage directory " 
         LOG.info("Completing previous upgrade for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         rename(getPreviousTmp(), getPreviousDir());
         rename(getPreviousTmp(), getPreviousDir());
         return;
         return;
       case RECOVER_UPGRADE:   // mv previous.tmp -> current
       case RECOVER_UPGRADE:   // mv previous.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous upgrade.");
+                 + " from previous upgrade");
         if (curDir.exists())
         if (curDir.exists())
           deleteDir(curDir);
           deleteDir(curDir);
         rename(getPreviousTmp(), curDir);
         rename(getPreviousTmp(), curDir);
         return;
         return;
       case COMPLETE_ROLLBACK: // rm removed.tmp
       case COMPLETE_ROLLBACK: // rm removed.tmp
         LOG.info("Completing previous rollback for storage directory "
         LOG.info("Completing previous rollback for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getRemovedTmp());
         deleteDir(getRemovedTmp());
         return;
         return;
       case RECOVER_ROLLBACK:  // mv removed.tmp -> current
       case RECOVER_ROLLBACK:  // mv removed.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous rollback.");
+                 + " from previous rollback");
         rename(getRemovedTmp(), curDir);
         rename(getRemovedTmp(), curDir);
         return;
         return;
       case COMPLETE_FINALIZE: // rm finalized.tmp
       case COMPLETE_FINALIZE: // rm finalized.tmp
         LOG.info("Completing previous finalize for storage directory "
         LOG.info("Completing previous finalize for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getFinalizedTmp());
         deleteDir(getFinalizedTmp());
         return;
         return;
       case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
       case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
         LOG.info("Completing previous checkpoint for storage directory " 
         LOG.info("Completing previous checkpoint for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         File prevCkptDir = getPreviousCheckpoint();
         File prevCkptDir = getPreviousCheckpoint();
         if (prevCkptDir.exists())
         if (prevCkptDir.exists())
           deleteDir(prevCkptDir);
           deleteDir(prevCkptDir);
@@ -574,7 +574,7 @@ public abstract class Storage extends StorageInfo {
         return;
         return;
       case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
       case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from failed checkpoint.");
+                 + " from failed checkpoint");
         if (curDir.exists())
         if (curDir.exists())
           deleteDir(curDir);
           deleteDir(curDir);
         rename(getLastCheckpointTmp(), curDir);
         rename(getLastCheckpointTmp(), curDir);
@@ -629,7 +629,7 @@ public abstract class Storage extends StorageInfo {
       FileLock newLock = tryLock();
       FileLock newLock = tryLock();
       if (newLock == null) {
       if (newLock == null) {
         String msg = "Cannot lock storage " + this.root 
         String msg = "Cannot lock storage " + this.root 
-          + ". The directory is already locked.";
+          + ". The directory is already locked";
         LOG.info(msg);
         LOG.info(msg);
         throw new IOException(msg);
         throw new IOException(msg);
       }
       }

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -75,14 +75,18 @@ class BPServiceActor implements Runnable {
 
 
   BPOfferService bpos;
   BPOfferService bpos;
   
   
-  long lastBlockReport = 0;
-  long lastDeletedReport = 0;
+  // lastBlockReport, lastDeletedReport and lastHeartbeat may be assigned/read
+  // by testing threads (through BPServiceActor#triggerXXX), while also 
+  // assigned/read by the actor thread. Thus they should be declared as volatile
+  // to make sure the "happens-before" consistency.
+  volatile long lastBlockReport = 0;
+  volatile long lastDeletedReport = 0;
 
 
   boolean resetBlockReportTime = true;
   boolean resetBlockReportTime = true;
 
 
   Thread bpThread;
   Thread bpThread;
   DatanodeProtocolClientSideTranslatorPB bpNamenode;
   DatanodeProtocolClientSideTranslatorPB bpNamenode;
-  private long lastHeartbeat = 0;
+  private volatile long lastHeartbeat = 0;
   private volatile boolean initialized = false;
   private volatile boolean initialized = false;
   
   
   /**
   /**
@@ -637,8 +641,7 @@ class BPServiceActor implements Runnable {
     try {
     try {
       Thread.sleep(millis);
       Thread.sleep(millis);
     } catch (InterruptedException ie) {
     } catch (InterruptedException ie) {
-      LOG.info("BPOfferService " + this +
-          " interrupted while " + stateString);
+      LOG.info("BPOfferService " + this + " interrupted while " + stateString);
     }
     }
   }
   }
 
 

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java

@@ -106,15 +106,15 @@ class BlockPoolManager {
     }
     }
   }
   }
   
   
-  void shutDownAll() throws InterruptedException {
-    BPOfferService[] bposArray = this.getAllNamenodeThreads();
-    
-    for (BPOfferService bpos : bposArray) {
-      bpos.stop(); //interrupts the threads
-    }
-    //now join
-    for (BPOfferService bpos : bposArray) {
-      bpos.join();
+  void shutDownAll(BPOfferService[] bposArray) throws InterruptedException {
+    if (bposArray != null) {
+      for (BPOfferService bpos : bposArray) {
+        bpos.stop(); //interrupts the threads
+      }
+      //now join
+      for (BPOfferService bpos : bposArray) {
+        bpos.join();
+      }
     }
     }
   }
   }
   
   

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -154,7 +154,7 @@ class BlockPoolSliceScanner {
     }
     }
     this.scanPeriod = hours * 3600 * 1000;
     this.scanPeriod = hours * 3600 * 1000;
     LOG.info("Periodic Block Verification Scanner initialized with interval "
     LOG.info("Periodic Block Verification Scanner initialized with interval "
-        + hours + " hours for block pool " + bpid + ".");
+        + hours + " hours for block pool " + bpid);
 
 
     // get the list of blocks and arrange them in random order
     // get the list of blocks and arrange them in random order
     List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
     List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
@@ -310,12 +310,12 @@ class BlockPoolSliceScanner {
   }
   }
   
   
   private void handleScanFailure(ExtendedBlock block) {
   private void handleScanFailure(ExtendedBlock block) {
-    LOG.info("Reporting bad block " + block);
+    LOG.info("Reporting bad " + block);
     try {
     try {
       datanode.reportBadBlocks(block);
       datanode.reportBadBlocks(block);
     } catch (IOException ie) {
     } catch (IOException ie) {
       // it is bad, but not bad enough to shutdown the scanner
       // it is bad, but not bad enough to shutdown the scanner
-      LOG.warn("Cannot report bad block=" + block.getBlockId());
+      LOG.warn("Cannot report bad " + block.getBlockId());
     }
     }
   }
   }
   
   
@@ -411,7 +411,7 @@ class BlockPoolSliceScanner {
 
 
         // If the block does not exists anymore, then its not an error
         // If the block does not exists anymore, then its not an error
         if (!dataset.contains(block)) {
         if (!dataset.contains(block)) {
-          LOG.info(block + " is no longer in the dataset.");
+          LOG.info(block + " is no longer in the dataset");
           deleteBlock(block.getLocalBlock());
           deleteBlock(block.getLocalBlock());
           return;
           return;
         }
         }
@@ -424,7 +424,7 @@ class BlockPoolSliceScanner {
         // is a block really deleted by mistake, DirectoryScan should catch it.
         // is a block really deleted by mistake, DirectoryScan should catch it.
         if (e instanceof FileNotFoundException ) {
         if (e instanceof FileNotFoundException ) {
           LOG.info("Verification failed for " + block +
           LOG.info("Verification failed for " + block +
-              ". It may be due to race with write.");
+              " - may be due to race with write");
           deleteBlock(block.getLocalBlock());
           deleteBlock(block.getLocalBlock());
           return;
           return;
         }
         }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -332,7 +332,7 @@ public class BlockPoolSliceStorage extends Storage {
     // 4.rename <SD>/curernt/<bpid>/previous.tmp to <SD>/curernt/<bpid>/previous
     // 4.rename <SD>/curernt/<bpid>/previous.tmp to <SD>/curernt/<bpid>/previous
     rename(bpTmpDir, bpPrevDir);
     rename(bpTmpDir, bpPrevDir);
     LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot()
     LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot()
-        + " is complete.");
+        + " is complete");
   }
   }
 
 
   /**
   /**
@@ -409,7 +409,7 @@ public class BlockPoolSliceStorage extends Storage {
     
     
     // 3. delete removed.tmp dir
     // 3. delete removed.tmp dir
     deleteDir(tmpDir);
     deleteDir(tmpDir);
-    LOG.info("Rollback of " + bpSd.getRoot() + " is complete.");
+    LOG.info("Rollback of " + bpSd.getRoot() + " is complete");
   }
   }
 
 
   /*
   /*

+ 12 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -357,7 +357,7 @@ class BlockReceiver implements Closeable {
   private void handleMirrorOutError(IOException ioe) throws IOException {
   private void handleMirrorOutError(IOException ioe) throws IOException {
     String bpid = block.getBlockPoolId();
     String bpid = block.getBlockPoolId();
     LOG.info(datanode.getDNRegistrationForBP(bpid)
     LOG.info(datanode.getDNRegistrationForBP(bpid)
-        + ":Exception writing block " + block + " to mirror " + mirrorAddr, ioe);
+        + ":Exception writing " + block + " to mirror " + mirrorAddr, ioe);
     if (Thread.interrupted()) { // shut down if the thread is interrupted
     if (Thread.interrupted()) { // shut down if the thread is interrupted
       throw ioe;
       throw ioe;
     } else { // encounter an error while writing to mirror
     } else { // encounter an error while writing to mirror
@@ -379,16 +379,16 @@ class BlockReceiver implements Closeable {
       LOG.warn("Checksum error in block " + block + " from " + inAddr, ce);
       LOG.warn("Checksum error in block " + block + " from " + inAddr, ce);
       if (srcDataNode != null) {
       if (srcDataNode != null) {
         try {
         try {
-          LOG.info("report corrupt block " + block + " from datanode " +
+          LOG.info("report corrupt " + block + " from datanode " +
                     srcDataNode + " to namenode");
                     srcDataNode + " to namenode");
           datanode.reportRemoteBadBlock(srcDataNode, block);
           datanode.reportRemoteBadBlock(srcDataNode, block);
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.warn("Failed to report bad block " + block + 
+          LOG.warn("Failed to report bad " + block + 
                     " from datanode " + srcDataNode + " to namenode");
                     " from datanode " + srcDataNode + " to namenode");
         }
         }
       }
       }
-      throw new IOException("Unexpected checksum mismatch " + 
-                            "while writing " + block + " from " + inAddr);
+      throw new IOException("Unexpected checksum mismatch while writing "
+          + block + " from " + inAddr);
     }
     }
   }
   }
   
   
@@ -518,7 +518,7 @@ class BlockReceiver implements Closeable {
           // If this is a partial chunk, then read in pre-existing checksum
           // If this is a partial chunk, then read in pre-existing checksum
           if (firstByteInBlock % bytesPerChecksum != 0) {
           if (firstByteInBlock % bytesPerChecksum != 0) {
             LOG.info("Packet starts at " + firstByteInBlock +
             LOG.info("Packet starts at " + firstByteInBlock +
-                     " for block " + block +
+                     " for " + block +
                      " which is not a multiple of bytesPerChecksum " +
                      " which is not a multiple of bytesPerChecksum " +
                      bytesPerChecksum);
                      bytesPerChecksum);
             long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
             long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
@@ -662,7 +662,7 @@ class BlockReceiver implements Closeable {
       }
       }
 
 
     } catch (IOException ioe) {
     } catch (IOException ioe) {
-      LOG.info("Exception in receiveBlock for " + block, ioe);
+      LOG.info("Exception for " + block, ioe);
       throw ioe;
       throw ioe;
     } finally {
     } finally {
       if (!responderClosed) { // Abnormal termination of the flow above
       if (!responderClosed) { // Abnormal termination of the flow above
@@ -733,10 +733,9 @@ class BlockReceiver implements Closeable {
     int checksumSize = diskChecksum.getChecksumSize();
     int checksumSize = diskChecksum.getChecksumSize();
     blkoff = blkoff - sizePartialChunk;
     blkoff = blkoff - sizePartialChunk;
     LOG.info("computePartialChunkCrc sizePartialChunk " + 
     LOG.info("computePartialChunkCrc sizePartialChunk " + 
-              sizePartialChunk +
-              " block " + block +
-              " offset in block " + blkoff +
-              " offset in metafile " + ckoff);
+              sizePartialChunk + " " + block +
+              " block offset " + blkoff +
+              " metafile offset " + ckoff);
 
 
     // create an input stream from the block file
     // create an input stream from the block file
     // and read in partial crc chunk into temporary buffer
     // and read in partial crc chunk into temporary buffer
@@ -758,7 +757,7 @@ class BlockReceiver implements Closeable {
     partialCrc = DataChecksum.newDataChecksum(
     partialCrc = DataChecksum.newDataChecksum(
         diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum());
         diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum());
     partialCrc.update(buf, 0, sizePartialChunk);
     partialCrc.update(buf, 0, sizePartialChunk);
-    LOG.info("Read in partial CRC chunk from disk for block " + block);
+    LOG.info("Read in partial CRC chunk from disk for " + block);
 
 
     // paranoia! verify that the pre-computed crc matches what we
     // paranoia! verify that the pre-computed crc matches what we
     // recalculated just now
     // recalculated just now
@@ -973,7 +972,7 @@ class BlockReceiver implements Closeable {
                       "HDFS_WRITE", clientname, offset,
                       "HDFS_WRITE", clientname, offset,
                       dnR.getStorageID(), block, endTime-startTime));
                       dnR.getStorageID(), block, endTime-startTime));
               } else {
               } else {
-                LOG.info("Received block " + block + " of size "
+                LOG.info("Received " + block + " size "
                     + block.getNumBytes() + " from " + inAddr);
                     + block.getNumBytes() + " from " + inAddr);
               }
               }
             }
             }

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

@@ -503,7 +503,7 @@ class BlockSender implements java.io.Closeable {
          * part of a block and then decides not to read the rest (but leaves
          * part of a block and then decides not to read the rest (but leaves
          * the socket open).
          * the socket open).
          */
          */
-          LOG.info("BlockSender.sendChunks() exception: ", e);
+          LOG.info("exception: ", e);
       } else {
       } else {
         /* Exception while writing to the client. Connection closure from
         /* Exception while writing to the client. Connection closure from
          * the other end is mostly the case and we do not care much about
          * the other end is mostly the case and we do not care much about

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -481,8 +481,7 @@ public class DataNode extends Configured
       blockScanner = new DataBlockScanner(this, data, conf);
       blockScanner = new DataBlockScanner(this, data, conf);
       blockScanner.start();
       blockScanner.start();
     } else {
     } else {
-      LOG.info("Periodic Block Verification scan is disabled because " +
-               reason + ".");
+      LOG.info("Periodic Block Verification scan disabled because " + reason);
     }
     }
   }
   }
   
   
@@ -511,7 +510,7 @@ public class DataNode extends Configured
       directoryScanner.start();
       directoryScanner.start();
     } else {
     } else {
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
       LOG.info("Periodic Directory Tree Verification scan is disabled because " +
-               reason + ".");
+               reason);
     }
     }
   }
   }
   
   
@@ -1095,6 +1094,12 @@ public class DataNode extends Configured
       }
       }
     }
     }
     
     
+    // We need to make a copy of the original blockPoolManager#offerServices to
+    // make sure blockPoolManager#shutDownAll() can still access all the 
+    // BPOfferServices, since after setting DataNode#shouldRun to false the 
+    // offerServices may be modified.
+    BPOfferService[] bposArray = this.blockPoolManager == null ? null
+        : this.blockPoolManager.getAllNamenodeThreads();
     this.shouldRun = false;
     this.shouldRun = false;
     shutdownPeriodicScanners();
     shutdownPeriodicScanners();
     
     
@@ -1141,7 +1146,7 @@ public class DataNode extends Configured
     
     
     if(blockPoolManager != null) {
     if(blockPoolManager != null) {
       try {
       try {
-        this.blockPoolManager.shutDownAll();
+        this.blockPoolManager.shutDownAll(bposArray);
       } catch (InterruptedException ie) {
       } catch (InterruptedException ie) {
         LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie);
         LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie);
       }
       }
@@ -1256,7 +1261,7 @@ public class DataNode extends Configured
           xfersBuilder.append(xferTargets[i]);
           xfersBuilder.append(xferTargets[i]);
           xfersBuilder.append(" ");
           xfersBuilder.append(" ");
         }
         }
-        LOG.info(bpReg + " Starting thread to transfer block " + 
+        LOG.info(bpReg + " Starting thread to transfer " + 
                  block + " to " + xfersBuilder);                       
                  block + " to " + xfersBuilder);                       
       }
       }
 
 
@@ -2043,7 +2048,7 @@ public class DataNode extends Configured
     ExtendedBlock block = rb.getBlock();
     ExtendedBlock block = rb.getBlock();
     DatanodeInfo[] targets = rb.getLocations();
     DatanodeInfo[] targets = rb.getLocations();
     
     
-    LOG.info(who + " calls recoverBlock(block=" + block
+    LOG.info(who + " calls recoverBlock(" + block
         + ", targets=[" + Joiner.on(", ").join(targets) + "]"
         + ", targets=[" + Joiner.on(", ").join(targets) + "]"
         + ", newGenerationStamp=" + rb.getNewGenerationStamp() + ")");
         + ", newGenerationStamp=" + rb.getNewGenerationStamp() + ")");
   }
   }

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -155,11 +155,11 @@ public class DataStorage extends Storage {
           break;
           break;
         case NON_EXISTENT:
         case NON_EXISTENT:
           // ignore this storage
           // ignore this storage
-          LOG.info("Storage directory " + dataDir + " does not exist.");
+          LOG.info("Storage directory " + dataDir + " does not exist");
           it.remove();
           it.remove();
           continue;
           continue;
         case NOT_FORMATTED: // format
         case NOT_FORMATTED: // format
-          LOG.info("Storage directory " + dataDir + " is not formatted.");
+          LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Formatting ...");
           LOG.info("Formatting ...");
           format(sd, nsInfo);
           format(sd, nsInfo);
           break;
           break;
@@ -482,7 +482,7 @@ public class DataStorage extends Storage {
     
     
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     rename(tmpDir, prevDir);
     rename(tmpDir, prevDir);
-    LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
+    LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
     addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage);
     addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage);
   }
   }
 
 
@@ -556,7 +556,7 @@ public class DataStorage extends Storage {
     rename(prevDir, curDir);
     rename(prevDir, curDir);
     // delete tmp dir
     // delete tmp dir
     deleteDir(tmpDir);
     deleteDir(tmpDir);
-    LOG.info("Rollback of " + sd.getRoot() + " is complete.");
+    LOG.info("Rollback of " + sd.getRoot() + " is complete");
   }
   }
   
   
   /**
   /**
@@ -596,9 +596,9 @@ public class DataStorage extends Storage {
               deleteDir(bbwDir);
               deleteDir(bbwDir);
             }
             }
           } catch(IOException ex) {
           } catch(IOException ex) {
-            LOG.error("Finalize upgrade for " + dataDirPath + " failed.", ex);
+            LOG.error("Finalize upgrade for " + dataDirPath + " failed", ex);
           }
           }
-          LOG.info("Finalize upgrade for " + dataDirPath + " is complete.");
+          LOG.info("Finalize upgrade for " + dataDirPath + " is complete");
         }
         }
         @Override
         @Override
         public String toString() { return "Finalize " + dataDirPath; }
         public String toString() { return "Finalize " + dataDirPath; }

+ 9 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -170,7 +170,7 @@ class DataXceiver extends Receiver implements Runnable {
         } catch (InvalidMagicNumberException imne) {
         } catch (InvalidMagicNumberException imne) {
           LOG.info("Failed to read expected encryption handshake from client " +
           LOG.info("Failed to read expected encryption handshake from client " +
               "at " + s.getInetAddress() + ". Perhaps the client is running an " +
               "at " + s.getInetAddress() + ". Perhaps the client is running an " +
-              "older version of Hadoop which does not support encryption.");
+              "older version of Hadoop which does not support encryption");
           return;
           return;
         }
         }
         input = encryptedStreams.in;
         input = encryptedStreams.in;
@@ -367,9 +367,8 @@ class DataXceiver extends Receiver implements Runnable {
     // make a copy here.
     // make a copy here.
     final ExtendedBlock originalBlock = new ExtendedBlock(block);
     final ExtendedBlock originalBlock = new ExtendedBlock(block);
     block.setNumBytes(dataXceiverServer.estimateBlockSize);
     block.setNumBytes(dataXceiverServer.estimateBlockSize);
-    LOG.info("Receiving block " + block + 
-             " src: " + remoteAddress +
-             " dest: " + localAddress);
+    LOG.info("Receiving " + block + " src: " + remoteAddress + " dest: "
+        + localAddress);
 
 
     // reply to upstream datanode or client 
     // reply to upstream datanode or client 
     final DataOutputStream replyOut = new DataOutputStream(
     final DataOutputStream replyOut = new DataOutputStream(
@@ -478,9 +477,9 @@ class DataXceiver extends Receiver implements Runnable {
                       block + " to mirror " + mirrorNode + ": " + e);
                       block + " to mirror " + mirrorNode + ": " + e);
             throw e;
             throw e;
           } else {
           } else {
-            LOG.info(datanode + ":Exception transfering block " +
+            LOG.info(datanode + ":Exception transfering " +
                      block + " to mirror " + mirrorNode +
                      block + " to mirror " + mirrorNode +
-                     ". continuing without the mirror.", e);
+                     "- continuing without the mirror", e);
           }
           }
         }
         }
       }
       }
@@ -528,10 +527,8 @@ class DataXceiver extends Receiver implements Runnable {
       if (isDatanode ||
       if (isDatanode ||
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
         datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
         datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
-        LOG.info("Received block " + block + 
-                 " src: " + remoteAddress +
-                 " dest: " + localAddress +
-                 " of size " + block.getNumBytes());
+        LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
+            + localAddress + " of size " + block.getNumBytes());
       }
       }
 
 
       
       
@@ -674,7 +671,7 @@ class DataXceiver extends Receiver implements Runnable {
       datanode.metrics.incrBytesRead((int) read);
       datanode.metrics.incrBytesRead((int) read);
       datanode.metrics.incrBlocksRead();
       datanode.metrics.incrBlocksRead();
       
       
-      LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
+      LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress());
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       isOpSuccess = false;
       isOpSuccess = false;
       LOG.info("opCopyBlock " + block + " received exception " + ioe);
       LOG.info("opCopyBlock " + block + " received exception " + ioe);
@@ -797,8 +794,7 @@ class DataXceiver extends Receiver implements Runnable {
       // notify name node
       // notify name node
       datanode.notifyNamenodeReceivedBlock(block, delHint);
       datanode.notifyNamenodeReceivedBlock(block, delHint);
 
 
-      LOG.info("Moved block " + block + 
-          " from " + s.getRemoteSocketAddress());
+      LOG.info("Moved " + block + " from " + s.getRemoteSocketAddress());
       
       
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       opStatus = ERROR;
       opStatus = ERROR;

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java

@@ -136,7 +136,7 @@ class FsDatasetAsyncDiskService {
     if (executors == null) {
     if (executors == null) {
       LOG.warn("AsyncDiskService has already shut down.");
       LOG.warn("AsyncDiskService has already shut down.");
     } else {
     } else {
-      LOG.info("Shutting down all async disk service threads...");
+      LOG.info("Shutting down all async disk service threads");
       
       
       for (Map.Entry<File, ThreadPoolExecutor> e : executors.entrySet()) {
       for (Map.Entry<File, ThreadPoolExecutor> e : executors.entrySet()) {
         e.getValue().shutdown();
         e.getValue().shutdown();
@@ -144,7 +144,7 @@ class FsDatasetAsyncDiskService {
       // clear the executor map so that calling execute again will fail.
       // clear the executor map so that calling execute again will fail.
       executors = null;
       executors = null;
       
       
-      LOG.info("All async disk service threads have been shut down.");
+      LOG.info("All async disk service threads have been shut down");
     }
     }
   }
   }
 
 
@@ -154,7 +154,7 @@ class FsDatasetAsyncDiskService {
    */
    */
   void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
   void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile,
       ExtendedBlock block) {
       ExtendedBlock block) {
-    LOG.info("Scheduling block " + block.getLocalBlock()
+    LOG.info("Scheduling " + block.getLocalBlock()
         + " file " + blockFile + " for deletion");
         + " file " + blockFile + " for deletion");
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
         volume, blockFile, metaFile, block);
         volume, blockFile, metaFile, block);
@@ -198,8 +198,8 @@ class FsDatasetAsyncDiskService {
           datanode.notifyNamenodeDeletedBlock(block);
           datanode.notifyNamenodeDeletedBlock(block);
         }
         }
         volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
         volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
-        LOG.info("Deleted block " + block.getBlockPoolId() + " "
-            + block.getLocalBlock() + " at file " + blockFile);
+        LOG.info("Deleted " + block.getBlockPoolId() + " "
+            + block.getLocalBlock() + " file " + blockFile);
       }
       }
     }
     }
   }
   }

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

@@ -425,7 +425,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       return;
       return;
     }
     }
     if (newlen > oldlen) {
     if (newlen > oldlen) {
-      throw new IOException("Cannout truncate block to from oldlen (=" + oldlen
+      throw new IOException("Cannot truncate block to from oldlen (=" + oldlen
           + ") to newlen (=" + newlen + ")");
           + ") to newlen (=" + newlen + ")");
     }
     }
 
 
@@ -481,7 +481,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           " should be greater than the replica " + b + "'s generation stamp");
           " should be greater than the replica " + b + "'s generation stamp");
     }
     }
     ReplicaInfo replicaInfo = getReplicaInfo(b);
     ReplicaInfo replicaInfo = getReplicaInfo(b);
-    LOG.info("Appending to replica " + replicaInfo);
+    LOG.info("Appending to " + replicaInfo);
     if (replicaInfo.getState() != ReplicaState.FINALIZED) {
     if (replicaInfo.getState() != ReplicaState.FINALIZED) {
       throw new ReplicaNotFoundException(
       throw new ReplicaNotFoundException(
           ReplicaNotFoundException.UNFINALIZED_REPLICA + b);
           ReplicaNotFoundException.UNFINALIZED_REPLICA + b);
@@ -689,7 +689,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b,
   public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd)
       long newGS, long minBytesRcvd, long maxBytesRcvd)
       throws IOException {
       throws IOException {
-    LOG.info("Recover the RBW replica " + b);
+    LOG.info("Recover RBW replica " + b);
 
 
     ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId());
     ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId());
     
     
@@ -700,7 +700,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
     }
     ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo;
     ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo;
     
     
-    LOG.info("Recovering replica " + rbw);
+    LOG.info("Recovering " + rbw);
 
 
     // Stop the previous writer
     // Stop the previous writer
     rbw.stopWriter();
     rbw.stopWriter();
@@ -736,8 +736,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final long blockId = b.getBlockId();
     final long blockId = b.getBlockId();
     final long expectedGs = b.getGenerationStamp();
     final long expectedGs = b.getGenerationStamp();
     final long visible = b.getNumBytes();
     final long visible = b.getNumBytes();
-    LOG.info("Convert replica " + b
-        + " from Temporary to RBW, visible length=" + visible);
+    LOG.info("Convert " + b + " from Temporary to RBW, visible length="
+        + visible);
 
 
     final ReplicaInPipeline temp;
     final ReplicaInPipeline temp;
     {
     {
@@ -1415,8 +1415,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   static ReplicaRecoveryInfo initReplicaRecovery(String bpid,
   static ReplicaRecoveryInfo initReplicaRecovery(String bpid,
       ReplicaMap map, Block block, long recoveryId) throws IOException {
       ReplicaMap map, Block block, long recoveryId) throws IOException {
     final ReplicaInfo replica = map.get(bpid, block.getBlockId());
     final ReplicaInfo replica = map.get(bpid, block.getBlockId());
-    LOG.info("initReplicaRecovery: block=" + block
-        + ", recoveryId=" + recoveryId
+    LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId
         + ", replica=" + replica);
         + ", replica=" + replica);
 
 
     //check replica
     //check replica
@@ -1485,7 +1484,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     //get replica
     //get replica
     final String bpid = oldBlock.getBlockPoolId();
     final String bpid = oldBlock.getBlockPoolId();
     final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId());
     final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId());
-    LOG.info("updateReplica: block=" + oldBlock
+    LOG.info("updateReplica: " + oldBlock
         + ", recoveryId=" + recoveryId
         + ", recoveryId=" + recoveryId
         + ", length=" + newlength
         + ", length=" + newlength
         + ", replica=" + replica);
         + ", replica=" + replica);

+ 5 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -142,7 +142,7 @@ public class FSDirectory implements Closeable {
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
     NameNode.LOG.info("Caching file names occuring more than " + threshold
     NameNode.LOG.info("Caching file names occuring more than " + threshold
-        + " times ");
+        + " times");
     nameCache = new NameCache<ByteArray>(threshold);
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;
     namesystem = ns;
   }
   }
@@ -255,15 +255,12 @@ public class FSDirectory implements Closeable {
       writeUnlock();
       writeUnlock();
     }
     }
     if (newNode == null) {
     if (newNode == null) {
-      NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
-                                   +"failed to add "+path
-                                   +" to the file system");
+      NameNode.stateChangeLog.info("DIR* addFile: failed to add " + path);
       return null;
       return null;
     }
     }
 
 
     if(NameNode.stateChangeLog.isDebugEnabled()) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
-          +path+" is added to the file system");
+      NameNode.stateChangeLog.debug("DIR* addFile: " + path + " is added");
     }
     }
     return newNode;
     return newNode;
   }
   }
@@ -2189,16 +2186,13 @@ public class FSDirectory implements Closeable {
       writeUnlock();
       writeUnlock();
     }
     }
     if (newNode == null) {
     if (newNode == null) {
-      NameNode.stateChangeLog.info("DIR* FSDirectory.addSymlink: "
-                                   +"failed to add "+path
-                                   +" to the file system");
+      NameNode.stateChangeLog.info("DIR* addSymlink: failed to add " + path);
       return null;
       return null;
     }
     }
     fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode);
     fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode);
     
     
     if(NameNode.stateChangeLog.isDebugEnabled()) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* FSDirectory.addSymlink: "
-          +path+" is added to the file system");
+      NameNode.stateChangeLog.debug("DIR* addSymlink: " + path + " is added");
     }
     }
     return newNode;
     return newNode;
   }
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -928,7 +928,7 @@ public class FSEditLog implements LogsPurgeable {
    * in the new log.
    * in the new log.
    */
    */
   synchronized long rollEditLog() throws IOException {
   synchronized long rollEditLog() throws IOException {
-    LOG.info("Rolling edit logs.");
+    LOG.info("Rolling edit logs");
     endCurrentLogSegment(true);
     endCurrentLogSegment(true);
     
     
     long nextTxId = getLastWrittenTxId() + 1;
     long nextTxId = getLastWrittenTxId() + 1;
@@ -943,7 +943,7 @@ public class FSEditLog implements LogsPurgeable {
    */
    */
   public synchronized void startLogSegment(long txid, 
   public synchronized void startLogSegment(long txid, 
       boolean abortCurrentLogSegment) throws IOException {
       boolean abortCurrentLogSegment) throws IOException {
-    LOG.info("Namenode started a new log segment at txid " + txid);
+    LOG.info("Started a new log segment at txid " + txid);
     if (isSegmentOpen()) {
     if (isSegmentOpen()) {
       if (getLastWrittenTxId() == txid - 1) {
       if (getLastWrittenTxId() == txid - 1) {
         //In sync with the NN, so end and finalize the current segment`
         //In sync with the NN, so end and finalize the current segment`

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -90,7 +90,7 @@ public class FSEditLogLoader {
                                  expectedStartingTxId, recovery);
                                  expectedStartingTxId, recovery);
       FSImage.LOG.info("Edits file " + edits.getName() 
       FSImage.LOG.info("Edits file " + edits.getName() 
           + " of size " + edits.length() + " edits # " + numEdits 
           + " of size " + edits.length() + " edits # " + numEdits 
-          + " loaded in " + (now()-startTime)/1000 + " seconds.");
+          + " loaded in " + (now()-startTime)/1000 + " seconds");
       return numEdits;
       return numEdits;
     } finally {
     } finally {
       edits.close();
       edits.close();

+ 26 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -660,11 +660,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         editLog.recoverUnclosedStreams();
         editLog.recoverUnclosedStreams();
         
         
         LOG.info("Catching up to latest edits from old active before " +
         LOG.info("Catching up to latest edits from old active before " +
-            "taking over writer role in edits logs.");
+            "taking over writer role in edits logs");
         editLogTailer.catchupDuringFailover();
         editLogTailer.catchupDuringFailover();
         blockManager.setPostponeBlocksFromFuture(false);
         blockManager.setPostponeBlocksFromFuture(false);
         
         
-        LOG.info("Reprocessing replication and invalidation queues...");
+        LOG.info("Reprocessing replication and invalidation queues");
         blockManager.getDatanodeManager().markAllDatanodesStale();
         blockManager.getDatanodeManager().markAllDatanodesStale();
         blockManager.clearQueues();
         blockManager.clearQueues();
         blockManager.processAllPendingDNMessages();
         blockManager.processAllPendingDNMessages();
@@ -1978,7 +1978,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (force) {
       if (force) {
         // close now: no need to wait for soft lease expiration and 
         // close now: no need to wait for soft lease expiration and 
         // close only the file src
         // close only the file src
-        LOG.info("recoverLease: recover lease " + lease + ", src=" + src +
+        LOG.info("recoverLease: " + lease + ", src=" + src +
           " from client " + pendingFile.getClientName());
           " from client " + pendingFile.getClientName());
         internalReleaseLease(lease, src, holder);
         internalReleaseLease(lease, src, holder);
       } else {
       } else {
@@ -1990,8 +1990,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // period, then start lease recovery.
         // period, then start lease recovery.
         //
         //
         if (lease.expiredSoftLimit()) {
         if (lease.expiredSoftLimit()) {
-          LOG.info("startFile: recover lease " + lease + ", src=" + src +
-              " from client " + pendingFile.getClientName());
+          LOG.info("startFile: recover " + lease + ", src=" + src + " client "
+              + pendingFile.getClientName());
           boolean isClosed = internalReleaseLease(lease, src, null);
           boolean isClosed = internalReleaseLease(lease, src, null);
           if(!isClosed)
           if(!isClosed)
             throw new RecoveryInProgressException(
             throw new RecoveryInProgressException(
@@ -2167,7 +2167,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           }
           }
 
 
           // The retry case ("b" above) -- abandon the old block.
           // The retry case ("b" above) -- abandon the old block.
-          NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: " +
+          NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
               "caught retry for allocation of a new block in " +
               "caught retry for allocation of a new block in " +
               src + ". Abandoning old block " + lastBlockInFile);
               src + ". Abandoning old block " + lastBlockInFile);
           dir.removeBlock(src, pendingFile, lastBlockInFile);
           dir.removeBlock(src, pendingFile, lastBlockInFile);
@@ -2403,10 +2403,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // See HDFS-3031.
         // See HDFS-3031.
         final Block realLastBlock = ((INodeFile)inode).getLastBlock();
         final Block realLastBlock = ((INodeFile)inode).getLastBlock();
         if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
         if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
-          NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: " +
-              "received request from " + holder + " to complete file " + src +
+          NameNode.stateChangeLog.info("DIR* completeFile: " +
+              "request from " + holder + " to complete " + src +
               " which is already closed. But, it appears to be an RPC " +
               " which is already closed. But, it appears to be an RPC " +
-              "retry. Returning success.");
+              "retry. Returning success");
           return true;
           return true;
         }
         }
       }
       }
@@ -2421,8 +2421,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
     finalizeINodeFileUnderConstruction(src, pendingFile);
     finalizeINodeFileUnderConstruction(src, pendingFile);
 
 
-    NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: file " + src
-                                  + " is closed by " + holder);
+    NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
+        + holder);
     return true;
     return true;
   }
   }
 
 
@@ -2447,8 +2447,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     nextGenerationStamp();
     nextGenerationStamp();
     b.setGenerationStamp(getGenerationStamp());
     b.setGenerationStamp(getGenerationStamp());
     b = dir.addBlock(src, inodes, b, targets);
     b = dir.addBlock(src, inodes, b, targets);
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
-                                 +src+ ". " + blockPoolId + " "+ b);
+    NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+        + blockPoolId + " " + b);
     return b;
     return b;
   }
   }
 
 
@@ -2466,8 +2466,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         //
         //
         for (BlockInfo block: v.getBlocks()) {
         for (BlockInfo block: v.getBlocks()) {
           if (!block.isComplete()) {
           if (!block.isComplete()) {
-            LOG.info("BLOCK* NameSystem.checkFileProgress: "
-                + "block " + block + " has not reached minimal replication "
+            LOG.info("BLOCK* checkFileProgress: " + block
+                + " has not reached minimal replication "
                 + blockManager.minReplication);
                 + blockManager.minReplication);
             return false;
             return false;
           }
           }
@@ -2478,8 +2478,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         //
         //
         BlockInfo b = v.getPenultimateBlock();
         BlockInfo b = v.getPenultimateBlock();
         if (b != null && !b.isComplete()) {
         if (b != null && !b.isComplete()) {
-          LOG.info("BLOCK* NameSystem.checkFileProgress: "
-              + "block " + b + " has not reached minimal replication "
+          LOG.info("BLOCK* checkFileProgress: " + b
+              + " has not reached minimal replication "
               + blockManager.minReplication);
               + blockManager.minReplication);
           return false;
           return false;
         }
         }
@@ -2952,8 +2952,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   void fsync(String src, String clientName) 
   void fsync(String src, String clientName) 
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
-                                  + src + " for " + clientName);
+    NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
@@ -2984,7 +2983,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   boolean internalReleaseLease(Lease lease, String src, 
   boolean internalReleaseLease(Lease lease, String src, 
       String recoveryLeaseHolder) throws AlreadyBeingCreatedException, 
       String recoveryLeaseHolder) throws AlreadyBeingCreatedException, 
       IOException, UnresolvedLinkException {
       IOException, UnresolvedLinkException {
-    LOG.info("Recovering lease=" + lease + ", src=" + src);
+    LOG.info("Recovering " + lease + ", src=" + src);
     assert !isInSafeMode();
     assert !isInSafeMode();
     assert hasWriteLock();
     assert hasWriteLock();
 
 
@@ -3625,7 +3624,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                               "in order to create namespace image.");
                               "in order to create namespace image.");
       }
       }
       getFSImage().saveNamespace(this);
       getFSImage().saveNamespace(this);
-      LOG.info("New namespace image has been created.");
+      LOG.info("New namespace image has been created");
     } finally {
     } finally {
       readUnlock();
       readUnlock();
     }
     }
@@ -3843,11 +3842,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       }
       long timeInSafemode = now() - startTime;
       long timeInSafemode = now() - startTime;
       NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
       NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
-                                    + timeInSafemode/1000 + " secs.");
+                                    + timeInSafemode/1000 + " secs");
       NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
       NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
       
       
       if (reached >= 0) {
       if (reached >= 0) {
-        NameNode.stateChangeLog.info("STATE* Safe mode is OFF."); 
+        NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 
       }
       }
       reached = -1;
       reached = -1;
       safeMode = null;
       safeMode = null;
@@ -4167,7 +4166,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         }
         }
       }
       }
       if (!fsRunning) {
       if (!fsRunning) {
-        LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread. ");
+        LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
       } else {
       } else {
         // leave safe mode and stop the monitor
         // leave safe mode and stop the monitor
         leaveSafeMode();
         leaveSafeMode();
@@ -4352,7 +4351,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isEditlogOpenForWrite) {
       if (isEditlogOpenForWrite) {
         getEditLog().logSyncAll();
         getEditLog().logSyncAll();
       }
       }
-      NameNode.stateChangeLog.info("STATE* Safe mode is ON. "
+      NameNode.stateChangeLog.info("STATE* Safe mode is ON"
           + safeMode.getTurnOffTip());
           + safeMode.getTurnOffTip());
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -4367,7 +4366,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     writeLock();
     try {
     try {
       if (!isInSafeMode()) {
       if (!isInSafeMode()) {
-        NameNode.stateChangeLog.info("STATE* Safe mode is already OFF."); 
+        NameNode.stateChangeLog.info("STATE* Safe mode is already OFF"); 
         return;
         return;
       }
       }
       safeMode.leave();
       safeMode.leave();
@@ -4721,7 +4720,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
       
       
-      NameNode.stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
+      NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
       for (int i = 0; i < blocks.length; i++) {
       for (int i = 0; i < blocks.length; i++) {
         ExtendedBlock blk = blocks[i].getBlock();
         ExtendedBlock blk = blocks[i].getBlock();
         DatanodeInfo[] nodes = blocks[i].getLocations();
         DatanodeInfo[] nodes = blocks[i].getLocations();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java

@@ -77,7 +77,7 @@ public class GetDelegationTokenServlet extends DfsServlet {
       });
       });
 
 
     } catch(Exception e) {
     } catch(Exception e) {
-      LOG.info("Exception while sending token. Re-throwing. ", e);
+      LOG.info("Exception while sending token. Re-throwing ", e);
       resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
       resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
     } finally {
     } finally {
       if(dos != null) dos.close();
       if(dos != null) dos.close();

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -429,7 +429,7 @@ public class LeaseManager {
         return;
         return;
       }
       }
 
 
-      LOG.info("Lease " + oldest + " has expired hard limit");
+      LOG.info(oldest + " has expired hard limit");
 
 
       final List<String> removing = new ArrayList<String>();
       final List<String> removing = new ArrayList<String>();
       // need to create a copy of the oldest lease paths, becuase 
       // need to create a copy of the oldest lease paths, becuase 
@@ -441,15 +441,14 @@ public class LeaseManager {
       for(String p : leasePaths) {
       for(String p : leasePaths) {
         try {
         try {
           if(fsnamesystem.internalReleaseLease(oldest, p, HdfsServerConstants.NAMENODE_LEASE_HOLDER)) {
           if(fsnamesystem.internalReleaseLease(oldest, p, HdfsServerConstants.NAMENODE_LEASE_HOLDER)) {
-            LOG.info("Lease recovery for file " + p +
-                          " is complete. File closed.");
+            LOG.info("Lease recovery for " + p + " is complete. File closed.");
             removing.add(p);
             removing.add(p);
           } else {
           } else {
-            LOG.info("Started block recovery for file " + p +
-                          " lease " + oldest);
+            LOG.info("Started block recovery " + p + " lease " + oldest);
           }
           }
         } catch (IOException e) {
         } catch (IOException e) {
-          LOG.error("Cannot release the path "+p+" in the lease "+oldest, e);
+          LOG.error("Cannot release the path " + p + " in the lease "
+              + oldest, e);
           removing.add(p);
           removing.add(p);
         }
         }
       }
       }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java

@@ -102,7 +102,7 @@ public final class MetaRecoveryContext  {
       "without prompting. " + 
       "without prompting. " + 
       "(c/s/q/a)\n", "c", "s", "q", "a");
       "(c/s/q/a)\n", "c", "s", "q", "a");
     if (answer.equals("c")) {
     if (answer.equals("c")) {
-      LOG.info("Continuing.");
+      LOG.info("Continuing");
       return;
       return;
     } else if (answer.equals("s")) {
     } else if (answer.equals("s")) {
       throw new RequestStopException("user requested stop");
       throw new RequestStopException("user requested stop");

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/jni_helper.c

@@ -146,6 +146,7 @@ static int hashTableInit(void)
             if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
             if (hcreate(MAX_HASH_TABLE_ELEM) == 0) {
                 fprintf(stderr, "error creating hashtable, <%d>: %s\n",
                 fprintf(stderr, "error creating hashtable, <%d>: %s\n",
                         errno, strerror(errno));
                         errno, strerror(errno));
+                UNLOCK_HASH_TABLE();
                 return 0;
                 return 0;
             } 
             } 
             hashTableInited = 1;
             hashTableInited = 1;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientDatanodeProtocol.proto

@@ -23,6 +23,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "ClientDatanodeProtocolProtos";
 option java_outer_classname = "ClientDatanodeProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "ClientNamenodeProtocolProtos";
 option java_outer_classname = "ClientNamenodeProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -23,6 +23,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "DatanodeProtocolProtos";
 option java_outer_classname = "DatanodeProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/GetUserMappingsProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "GetUserMappingsProtocolProtos";
 option java_outer_classname = "GetUserMappingsProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 /**
 /**
  *  Get groups for user request.
  *  Get groups for user request.

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HAZKInfo.proto

@@ -17,6 +17,7 @@
  */
  */
 option java_package = "org.apache.hadoop.hdfs.server.namenode.ha.proto";
 option java_package = "org.apache.hadoop.hdfs.server.namenode.ha.proto";
 option java_outer_classname = "HAZKInfoProtos";
 option java_outer_classname = "HAZKInfoProtos";
+package hadoop.hdfs;
 
 
 message ActiveNodeInfo {
 message ActiveNodeInfo {
   required string nameserviceId = 1;
   required string nameserviceId = 1;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto

@@ -23,6 +23,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "InterDatanodeProtocolProtos";
 option java_outer_classname = "InterDatanodeProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto

@@ -23,6 +23,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "JournalProtocolProtos";
 option java_outer_classname = "JournalProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto

@@ -23,6 +23,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "NamenodeProtocolProtos";
 option java_outer_classname = "NamenodeProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.hdfs.qjournal.protocol";
 option java_outer_classname = "QJournalProtocolProtos";
 option java_outer_classname = "QJournalProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshAuthorizationPolicyProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "RefreshAuthorizationPolicyProtocolProtos";
 option java_outer_classname = "RefreshAuthorizationPolicyProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 /**
 /**
  *  Refresh service acl request.
  *  Refresh service acl request.

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RefreshUserMappingsProtocol.proto

@@ -20,6 +20,7 @@ option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "RefreshUserMappingsProtocolProtos";
 option java_outer_classname = "RefreshUserMappingsProtocolProtos";
 option java_generic_services = true;
 option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 /**
 /**
  *  Refresh user to group mappings request.
  *  Refresh user to group mappings request.

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/datatransfer.proto

@@ -22,6 +22,7 @@
 option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "DataTransferProtos";
 option java_outer_classname = "DataTransferProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 import "hdfs.proto";
 import "hdfs.proto";
 
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto

@@ -22,6 +22,7 @@
 option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_package = "org.apache.hadoop.hdfs.protocol.proto";
 option java_outer_classname = "HdfsProtos";
 option java_outer_classname = "HdfsProtos";
 option java_generate_equals_and_hash = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 
 /**
 /**
  * Extended block idenfies a block
  * Extended block idenfies a block

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml

@@ -15797,7 +15797,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
-          <expected-output>Configured Capacity: [0-9]+ \([0-9]+\.[0-9]+ [BKMGT]+\)</expected-output>
+          <expected-output>Configured Capacity: [0-9]+ \([0-9\.]+ [BKMGT]+\)</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
@@ -15915,7 +15915,7 @@
       <comparators>
       <comparators>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>
-          <expected-output>Configured Capacity: [0-9]+ \([0-9]+\.[0-9]+ [BKMGT]+\)</expected-output>
+          <expected-output>Configured Capacity: [0-9]+ \([0-9\.]+ [BKMGT]+\)</expected-output>
         </comparator>
         </comparator>
         <comparator>
         <comparator>
           <type>RegexpComparator</type>
           <type>RegexpComparator</type>

+ 2 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -613,6 +613,8 @@ Release 0.23.5 - UNRELEASED
     MAPREDUCE-4730. Fix Reducer's EventFetcher to scale the map-completion
     MAPREDUCE-4730. Fix Reducer's EventFetcher to scale the map-completion
     requests slowly to avoid HADOOP-8942. (Jason Lowe via vinodkv)
     requests slowly to avoid HADOOP-8942. (Jason Lowe via vinodkv)
 
 
+    MAPREDUCE-4748. Invalid event: T_ATTEMPT_SUCCEEDED at SUCCEEDED. (jlowe)
+
 Release 0.23.4 - UNRELEASED
 Release 0.23.4 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 4 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -203,7 +203,10 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
     .addTransition(
     .addTransition(
         TaskStateInternal.SUCCEEDED, TaskStateInternal.SUCCEEDED,
         TaskStateInternal.SUCCEEDED, TaskStateInternal.SUCCEEDED,
         EnumSet.of(TaskEventType.T_ADD_SPEC_ATTEMPT,
         EnumSet.of(TaskEventType.T_ADD_SPEC_ATTEMPT,
-            TaskEventType.T_ATTEMPT_LAUNCHED))
+            TaskEventType.T_ATTEMPT_COMMIT_PENDING,
+            TaskEventType.T_ATTEMPT_LAUNCHED,
+            TaskEventType.T_ATTEMPT_SUCCEEDED,
+            TaskEventType.T_KILL))
 
 
     // Transitions from FAILED state        
     // Transitions from FAILED state        
     .addTransition(TaskStateInternal.FAILED, TaskStateInternal.FAILED,
     .addTransition(TaskStateInternal.FAILED, TaskStateInternal.FAILED,

+ 34 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java

@@ -418,6 +418,21 @@ public class TestTaskImpl {
     killRunningTaskAttempt(getLastAttempt().getAttemptId());    
     killRunningTaskAttempt(getLastAttempt().getAttemptId());    
   }
   }
 
 
+  @Test
+  public void testKillSuccessfulTask() {
+    LOG.info("--- START: testKillSuccesfulTask ---");
+    mockTask = createMockTask(TaskType.MAP);
+    TaskId taskId = getNewTaskID();
+    scheduleTaskAttempt(taskId);
+    launchTaskAttempt(getLastAttempt().getAttemptId());
+    commitTaskAttempt(getLastAttempt().getAttemptId());
+    mockTask.handle(new TaskTAttemptEvent(getLastAttempt().getAttemptId(),
+        TaskEventType.T_ATTEMPT_SUCCEEDED));
+    assertTaskSucceededState();
+    mockTask.handle(new TaskEvent(taskId, TaskEventType.T_KILL));
+    assertTaskSucceededState();
+  }
+
   @Test 
   @Test 
   public void testTaskProgress() {
   public void testTaskProgress() {
     LOG.info("--- START: testTaskProgress ---");
     LOG.info("--- START: testTaskProgress ---");
@@ -485,7 +500,8 @@ public class TestTaskImpl {
     assertTaskSucceededState();
     assertTaskSucceededState();
   }
   }
   
   
-  private void runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType failEvent) {
+  private void runSpeculativeTaskAttemptSucceeds(
+      TaskEventType firstAttemptFinishEvent) {
     TaskId taskId = getNewTaskID();
     TaskId taskId = getNewTaskID();
     scheduleTaskAttempt(taskId);
     scheduleTaskAttempt(taskId);
     launchTaskAttempt(getLastAttempt().getAttemptId());
     launchTaskAttempt(getLastAttempt().getAttemptId());
@@ -502,9 +518,9 @@ public class TestTaskImpl {
     // The task should now have succeeded
     // The task should now have succeeded
     assertTaskSucceededState();
     assertTaskSucceededState();
     
     
-    // Now fail the first task attempt, after the second has succeeded
+    // Now complete the first task attempt, after the second has succeeded
     mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(), 
     mockTask.handle(new TaskTAttemptEvent(taskAttempts.get(0).getAttemptId(), 
-        failEvent));
+        firstAttemptFinishEvent));
     
     
     // The task should still be in the succeeded state
     // The task should still be in the succeeded state
     assertTaskSucceededState();
     assertTaskSucceededState();
@@ -513,25 +529,36 @@ public class TestTaskImpl {
   @Test
   @Test
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
     mockTask = createMockTask(TaskType.MAP);        
     mockTask = createMockTask(TaskType.MAP);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_FAILED);
   }
   }
 
 
   @Test
   @Test
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
     mockTask = createMockTask(TaskType.REDUCE);        
     mockTask = createMockTask(TaskType.REDUCE);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_FAILED);
   }
   }
   
   
   @Test
   @Test
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
     mockTask = createMockTask(TaskType.MAP);        
     mockTask = createMockTask(TaskType.MAP);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_KILLED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_KILLED);
   }
   }
 
 
   @Test
   @Test
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
     mockTask = createMockTask(TaskType.REDUCE);        
     mockTask = createMockTask(TaskType.REDUCE);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_KILLED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_KILLED);
   }
   }
 
 
+  @Test
+  public void testMultipleTaskAttemptsSucceed() {
+    mockTask = createMockTask(TaskType.MAP);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_SUCCEEDED);
+  }
+
+  @Test
+  public void testCommitAfterSucceeds() {
+    mockTask = createMockTask(TaskType.REDUCE);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_COMMIT_PENDING);
+  }
 }
 }

+ 2 - 0
hadoop-yarn-project/CHANGES.txt

@@ -23,6 +23,8 @@ Release 2.0.3-alpha - Unreleased
 
 
   NEW FEATURES
   NEW FEATURES
 
 
+    YARN-145. Add a Web UI to the fair share scheduler. (Sandy Ryza via tomwhite)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     YARN-78. Changed UnManagedAM application to use YarnClient. (Bikas Saha via
     YARN-78. Changed UnManagedAM application to use YarnClient. (Bikas Saha via

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -56,7 +56,9 @@ public class FSQueue {
 
 
   public void addApp(FSSchedulerApp app) {
   public void addApp(FSSchedulerApp app) {
     applications.add(app);
     applications.add(app);
-    queueSchedulable.addApp(new AppSchedulable(scheduler, app, this));
+    AppSchedulable appSchedulable = new AppSchedulable(scheduler, app, this);
+    app.setAppSchedulable(appSchedulable);
+    queueSchedulable.addApp(appSchedulable);
   }
   }
 
 
   public void removeJob(FSSchedulerApp app) {
   public void removeJob(FSSchedulerApp app) {

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java

@@ -70,6 +70,7 @@ public class FSSchedulerApp extends SchedulerApplication {
       .getRecordFactory(null);
       .getRecordFactory(null);
 
 
   private final AppSchedulingInfo appSchedulingInfo;
   private final AppSchedulingInfo appSchedulingInfo;
+  private AppSchedulable appSchedulable;
   private final Queue queue;
   private final Queue queue;
 
 
   private final Resource currentConsumption = recordFactory
   private final Resource currentConsumption = recordFactory
@@ -118,6 +119,14 @@ public class FSSchedulerApp extends SchedulerApplication {
   public ApplicationAttemptId getApplicationAttemptId() {
   public ApplicationAttemptId getApplicationAttemptId() {
     return this.appSchedulingInfo.getApplicationAttemptId();
     return this.appSchedulingInfo.getApplicationAttemptId();
   }
   }
+  
+  public void setAppSchedulable(AppSchedulable appSchedulable) {
+    this.appSchedulable = appSchedulable;
+  }
+  
+  public AppSchedulable getAppSchedulable() {
+    return appSchedulable;
+  }
 
 
   public String getUser() {
   public String getUser() {
     return this.appSchedulingInfo.getUser();
     return this.appSchedulingInfo.getUser();

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -788,7 +788,11 @@ public class FairScheduler implements ResourceScheduler {
     FSSchedulerNode node = nodes.get(nodeId);
     FSSchedulerNode node = nodes.get(nodeId);
     return node == null ? null : new SchedulerNodeReport(node);
     return node == null ? null : new SchedulerNodeReport(node);
   }
   }
-
+  
+  public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
+    return applications.get(appAttemptId);
+  }
+  
   @Override
   @Override
   public SchedulerAppReport getSchedulerAppInfo(
   public SchedulerAppReport getSchedulerAppInfo(
       ApplicationAttemptId appAttemptId) {
       ApplicationAttemptId appAttemptId) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -446,7 +446,7 @@ public class QueueManager {
    * Get the maximum resource allocation for the given queue.
    * Get the maximum resource allocation for the given queue.
    * @return the cap set on this queue, or Integer.MAX_VALUE if not set.
    * @return the cap set on this queue, or Integer.MAX_VALUE if not set.
    */
    */
-  Resource getMaxResources(String queueName) {
+  public Resource getMaxResources(String queueName) {
     synchronized (maxQueueResourcesMO) {
     synchronized (maxQueueResourcesMO) {
       if (maxQueueResources.containsKey(queueName)) {
       if (maxQueueResources.containsKey(queueName)) {
         return maxQueueResources.get(queueName);
         return maxQueueResources.get(queueName);

+ 136 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerAppsBlock.java

@@ -0,0 +1,136 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
+
+import java.util.Collection;
+import java.util.HashSet;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.JQueryUI.Render;
+
+import com.google.inject.Inject;
+
+/**
+ * Shows application information specific to the fair
+ * scheduler as part of the fair scheduler page.
+ */
+public class FairSchedulerAppsBlock extends HtmlBlock {
+  final AppsList list;
+  final FairSchedulerInfo fsinfo;
+  
+  @Inject public FairSchedulerAppsBlock(AppsList list, 
+      ResourceManager rm, ViewContext ctx) {
+    super(ctx);
+    this.list = list;
+    FairScheduler scheduler = (FairScheduler) rm.getResourceScheduler();
+    fsinfo = new FairSchedulerInfo(scheduler);
+  }
+  
+  @Override public void render(Block html) {
+    TBODY<TABLE<Hamlet>> tbody = html.
+      table("#apps").
+        thead().
+          tr().
+            th(".id", "ID").
+            th(".user", "User").
+            th(".name", "Name").
+            th(".queue", "Queue").
+            th(".fairshare", "Fair Share").
+            th(".starttime", "StartTime").
+            th(".finishtime", "FinishTime").
+            th(".state", "State").
+            th(".finalstatus", "FinalStatus").
+            th(".progress", "Progress").
+            th(".ui", "Tracking UI")._()._().
+        tbody();
+    int i = 0;
+    Collection<RMAppState> reqAppStates = null;
+    String reqStateString = $(APP_STATE);
+    if (reqStateString != null && !reqStateString.isEmpty()) {
+      String[] appStateStrings = reqStateString.split(",");
+      reqAppStates = new HashSet<RMAppState>(appStateStrings.length);
+      for(String stateString : appStateStrings) {
+        reqAppStates.add(RMAppState.valueOf(stateString));
+      }
+    }
+    for (RMApp app : list.apps.values()) {
+      if (reqAppStates != null && !reqAppStates.contains(app.getState())) {
+        continue;
+      }
+      AppInfo appInfo = new AppInfo(app, true);
+      String percent = String.format("%.1f", appInfo.getProgress());
+      String startTime = Times.format(appInfo.getStartTime());
+      String finishTime = Times.format(appInfo.getFinishTime());
+      ApplicationAttemptId attemptId = app.getCurrentAppAttempt().getAppAttemptId();
+      int fairShare = fsinfo.getAppFairShare(attemptId);
+
+      tbody.
+        tr().
+          td().
+            br().$title(appInfo.getAppIdNum())._(). // for sorting
+            a(url("app", appInfo.getAppId()), appInfo.getAppId())._().
+          td(appInfo.getUser()).
+          td(appInfo.getName()).
+          td(appInfo.getQueue()).
+          td("" + fairShare).
+          td().
+            br().$title(String.valueOf(appInfo.getStartTime()))._().
+            _(startTime)._().
+          td().
+            br().$title(String.valueOf(appInfo.getFinishTime()))._().
+            _(finishTime)._().
+          td(appInfo.getState()).
+          td(appInfo.getFinalStatus()).
+          td().
+            br().$title(percent)._(). // for sorting
+            div(_PROGRESSBAR).
+              $title(join(percent, '%')). // tooltip
+              div(_PROGRESSBAR_VALUE).
+                $style(join("width:", percent, '%'))._()._()._().
+          td().
+            a(!appInfo.isTrackingUrlReady()?
+              "#" : appInfo.getTrackingUrlPretty(), appInfo.getTrackingUI())._()._();
+      if (list.rendering != Render.HTML && ++i >= 20) break;
+    }
+    tbody._()._();
+
+    if (list.rendering == Render.JS_ARRAY) {
+      echo("<script type='text/javascript'>\n",
+           "var appsData=");
+      list.toDataTableArrays(reqAppStates, writer());
+      echo("\n</script>\n");
+    }
+  }
+}

+ 200 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java

@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerQueueInfo;
+import org.apache.hadoop.yarn.webapp.ResponseInfo;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+
+import com.google.inject.Inject;
+import com.google.inject.servlet.RequestScoped;
+
+public class FairSchedulerPage extends RmView {
+  static final String _Q = ".ui-state-default.ui-corner-all";
+  static final float Q_MAX_WIDTH = 0.8f;
+  static final float Q_STATS_POS = Q_MAX_WIDTH + 0.05f;
+  static final String Q_END = "left:101%";
+  static final String Q_GIVEN = "left:0%;background:none;border:1px dashed rgba(0,0,0,0.25)";
+  static final String Q_OVER = "background:rgba(255, 140, 0, 0.8)";
+  static final String Q_UNDER = "background:rgba(50, 205, 50, 0.8)";
+  
+  @RequestScoped
+  static class FSQInfo {
+    FairSchedulerInfo fsinfo;
+    FairSchedulerQueueInfo qinfo;
+  }
+  
+  static class QueueInfoBlock extends HtmlBlock {
+    final FairSchedulerQueueInfo qinfo;
+
+    @Inject QueueInfoBlock(ViewContext ctx, FSQInfo info) {
+      super(ctx);
+      qinfo = (FairSchedulerQueueInfo) info.qinfo;
+    }
+
+    @Override
+    protected void render(Block html) {
+      ResponseInfo ri = info("\'" + qinfo.getQueueName() + "\' Queue Status").
+          _("Used Resources:", qinfo.getUsedResources().toString()).
+          _("Num Active Applications:", qinfo.getNumActiveApplications()).
+          _("Num Pending Applications:", qinfo.getNumPendingApplications()).
+          _("Min Resources:", qinfo.getMinResources().toString()).
+          _("Max Resources:", qinfo.getMaxResources().toString());
+      int maxApps = qinfo.getMaxApplications();
+      if (maxApps < Integer.MAX_VALUE) {
+          ri._("Max Running Applications:", qinfo.getMaxApplications());
+      }
+      ri._("Fair Share:", qinfo.getFairShare());
+
+      html._(InfoBlock.class);
+
+      // clear the info contents so this queue's info doesn't accumulate into another queue's info
+      ri.clear();
+    }
+  }
+  
+  static class QueuesBlock extends HtmlBlock {
+    final FairScheduler fs;
+    final FSQInfo fsqinfo;
+    
+    @Inject QueuesBlock(ResourceManager rm, FSQInfo info) {
+      fs = (FairScheduler)rm.getResourceScheduler();
+      fsqinfo = info;
+    }
+    
+    @Override public void render(Block html) {
+      html._(MetricsOverviewTable.class);
+      UL<DIV<DIV<Hamlet>>> ul = html.
+        div("#cs-wrapper.ui-widget").
+          div(".ui-widget-header.ui-corner-top").
+            _("Application Queues")._().
+          div("#cs.ui-widget-content.ui-corner-bottom").
+            ul();
+      if (fs == null) {
+        ul.
+          li().
+            a(_Q).$style(width(Q_MAX_WIDTH)).
+              span().$style(Q_END)._("100% ")._().
+              span(".q", "default")._()._();
+      } else {
+        FairSchedulerInfo sinfo = new FairSchedulerInfo(fs);
+        fsqinfo.fsinfo = sinfo;
+        fsqinfo.qinfo = null;
+
+        ul.
+          li().$style("margin-bottom: 1em").
+            span().$style("font-weight: bold")._("Legend:")._().
+            span().$class("qlegend ui-corner-all").$style(Q_GIVEN).
+              _("Fair Share")._().
+            span().$class("qlegend ui-corner-all").$style(Q_UNDER).
+              _("Used")._().
+            span().$class("qlegend ui-corner-all").$style(Q_OVER).
+              _("Used (over fair share)")._().
+            span().$class("qlegend ui-corner-all ui-state-default").
+              _("Max Capacity")._().
+          _();
+        
+        List<FairSchedulerQueueInfo> subQueues = fsqinfo.fsinfo.getQueueInfos();
+        for (FairSchedulerQueueInfo info : subQueues) {
+          fsqinfo.qinfo = info;
+          float capacity = info.getMaxResourcesFraction();
+          float fairShare = info.getFairShareFraction();
+          float used = info.getUsedFraction();
+          ul.
+              li().
+                a(_Q).$style(width(capacity * Q_MAX_WIDTH)).
+                  $title(join("Fair Share:", percent(fairShare))).
+                  span().$style(join(Q_GIVEN, ";font-size:1px;", width(fairShare/capacity))).
+                    _('.')._().
+                  span().$style(join(width(used/capacity),
+                    ";font-size:1px;left:0%;", used > fairShare ? Q_OVER : Q_UNDER)).
+                    _('.')._().
+                  span(".q", info.getQueueName())._().
+                span().$class("qstats").$style(left(Q_STATS_POS)).
+                  _(join(percent(used), " used"))._().
+                ul("#lq").li()._(QueueInfoBlock.class)._()._().
+              _();
+        }
+      }
+      ul._()._().
+      script().$type("text/javascript").
+          _("$('#cs').hide();")._()._().
+      _(FairSchedulerAppsBlock.class);
+    }
+  }
+  
+  @Override protected void postHead(Page.HTML<_> html) {
+    html.
+      style().$type("text/css").
+        _("#cs { padding: 0.5em 0 1em 0; margin-bottom: 1em; position: relative }",
+          "#cs ul { list-style: none }",
+          "#cs a { font-weight: normal; margin: 2px; position: relative }",
+          "#cs a span { font-weight: normal; font-size: 80% }",
+          "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }",
+          "table.info tr th {width: 50%}")._(). // to center info table
+      script("/static/jt/jquery.jstree.js").
+      script().$type("text/javascript").
+        _("$(function() {",
+          "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
+          "  $('#cs').bind('loaded.jstree', function (e, data) {",
+          "    data.inst.open_all(); }).",
+          "    jstree({",
+          "    core: { animation: 188, html_titles: true },",
+          "    plugins: ['themeroller', 'html_data', 'ui'],",
+          "    themeroller: { item_open: 'ui-icon-minus',",
+          "      item_clsd: 'ui-icon-plus', item_leaf: 'ui-icon-gear'",
+          "    }",
+          "  });",
+          "  $('#cs').bind('select_node.jstree', function(e, data) {",
+          "    var q = $('.q', data.rslt.obj).first().text();",
+            "    if (q == 'root') q = '';",
+          "    $('#apps').dataTable().fnFilter(q, 3);",
+          "  });",
+          "  $('#cs').show();",
+          "});")._();
+  }
+  
+  @Override protected Class<? extends SubView> content() {
+    return QueuesBlock.class;
+  }
+
+  static String percent(float f) {
+    return String.format("%.1f%%", f * 100);
+  }
+
+  static String width(float f) {
+    return String.format("width:%.1f%%", f * 100);
+  }
+
+  static String left(float f) {
+    return String.format("left:%.1f%%", f * 100);
+  }
+}

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java

@@ -77,8 +77,9 @@ public class RmController extends Controller {
     }
     }
     
     
     if (rs instanceof FairScheduler) {
     if (rs instanceof FairScheduler) {
-      context().setStatus(404);
-      throw new WebAppException("Fair Scheduler UI not yet supported");
+      setTitle("Fair Scheduler");
+      render(FairSchedulerPage.class);
+      return;
     }
     }
     
     
     setTitle("Default Scheduler");
     setTitle("Default Scheduler");

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerInfo.java

@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+
+public class FairSchedulerInfo {
+  private List<FairSchedulerQueueInfo> queueInfos;
+  private FairScheduler scheduler;
+  
+  public FairSchedulerInfo(FairScheduler fs) {
+    scheduler = fs;
+    Collection<FSQueue> queues = fs.getQueueManager().getQueues();
+    queueInfos = new ArrayList<FairSchedulerQueueInfo>();
+    for (FSQueue queue : queues) {
+      queueInfos.add(new FairSchedulerQueueInfo(queue, fs));
+    }
+  }
+  
+  public List<FairSchedulerQueueInfo> getQueueInfos() {
+    return queueInfos;
+  }
+  
+  public int getAppFairShare(ApplicationAttemptId appAttemptId) {
+    return scheduler.getSchedulerApp(appAttemptId).
+        getAppSchedulable().getFairShare().getMemory();
+  }
+}

+ 153 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java

@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+
+import java.util.Collection;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueueSchedulable;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueueManager;
+
+public class FairSchedulerQueueInfo {
+  private int numPendingApps;
+  private int numActiveApps;
+  
+  private int fairShare;
+  private int minShare;
+  private int maxShare;
+  private int clusterMaxMem;
+  
+  private int maxApps;
+  
+  private float fractionUsed;
+  private float fractionFairShare;
+  private float fractionMinShare;
+  
+  private Resource minResources;
+  private Resource maxResources;
+  private Resource usedResources;
+  
+  private String queueName;
+  
+  public FairSchedulerQueueInfo(FSQueue queue, FairScheduler scheduler) {
+    Collection<FSSchedulerApp> apps = queue.getApplications();
+    for (FSSchedulerApp app : apps) {
+      if (app.isPending()) {
+        numPendingApps++;
+      } else {
+        numActiveApps++;
+      }
+    }
+    
+    FSQueueSchedulable schedulable = queue.getQueueSchedulable();
+    QueueManager manager = scheduler.getQueueManager();
+    
+    queueName = queue.getName();
+        
+    Resource clusterMax = scheduler.getClusterCapacity();
+    clusterMaxMem = clusterMax.getMemory();
+    
+    usedResources = schedulable.getResourceUsage();
+    fractionUsed = (float)usedResources.getMemory() / clusterMaxMem;
+    
+    fairShare = schedulable.getFairShare().getMemory();
+    minResources = schedulable.getMinShare();
+    minShare = minResources.getMemory();
+    maxResources = scheduler.getQueueManager().getMaxResources(queueName);
+    if (maxResources.getMemory() > clusterMaxMem) {
+      maxResources = Resources.createResource(clusterMaxMem);
+    }
+    maxShare = maxResources.getMemory();
+    
+    fractionFairShare = (float)fairShare / clusterMaxMem;
+    fractionMinShare = (float)minShare / clusterMaxMem;
+    
+    maxApps = manager.getQueueMaxApps(queueName);
+  }
+  
+  /**
+   * Returns the fair share as a fraction of the entire cluster capacity.
+   */
+  public float getFairShareFraction() {
+    return fractionFairShare;
+  }
+  
+  /**
+   * Returns the fair share of this queue in megabytes.
+   */
+  public int getFairShare() {
+    return fairShare;
+  }
+  
+  public int getNumActiveApplications() {
+    return numPendingApps;
+  }
+  
+  public int getNumPendingApplications() {
+    return numActiveApps;
+  }
+  
+  public Resource getMinResources() {
+    return minResources;
+  }
+  
+  public Resource getMaxResources() {
+    return maxResources;
+  }
+  
+  public int getMaxApplications() {
+    return maxApps;
+  }
+  
+  public String getQueueName() {
+    return queueName;
+  }
+  
+  public Resource getUsedResources() {
+    return usedResources;
+  }
+  
+  /**
+   * Returns the queue's min share in as a fraction of the entire
+   * cluster capacity.
+   */
+  public float getMinShareFraction() {
+    return fractionMinShare;
+  }
+  
+  /**
+   * Returns the memory used by this queue as a fraction of the entire 
+   * cluster capacity.
+   */
+  public float getUsedFraction() {
+    return fractionUsed;
+  }
+  
+  /**
+   * Returns the capacity of this queue as a fraction of the entire cluster 
+   * capacity.
+   */
+  public float getMaxResourcesFraction() {
+    return (float)maxShare / clusterMaxMem;
+  }
+}

部分文件因文件數量過多而無法顯示