瀏覽代碼

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-8985. Add namespace declarations in .proto files for languages 
+    other than java. (Binglin Chan via suresh)
+
   OPTIMIZATIONS
 
     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 
     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 
 
   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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 enum HAServiceStateProto {
   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_outer_classname = "IpcConnectionContextProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 /**
  * 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 /**
  * 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_outer_classname = "RpcPayloadHeaderProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 /**
  * 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 message CedeActiveRequestProto {
   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_outer_classname = "HadoopRpcProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 /**
  * 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.net.NetUtils;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 /** Unit test for supporting method-name based compatible RPCs. */
@@ -114,6 +115,11 @@ public class TestRPCCompatibility {
     }
 
   }
+
+  @Before
+  public void setUp() {
+    ProtocolSignature.resetCache();
+  }
   
   @After
   public void tearDown() throws IOException {
@@ -219,7 +225,6 @@ System.out.println("echo int is NOT supported");
   
   @Test // equal version client and server
   public void testVersion2ClientVersion2Server() throws Exception {
-    ProtocolSignature.resetCache();
     // create a server with two handlers
     TestImpl2 impl = new TestImpl2();
     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_outer_classname = "TestProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.common;
 
 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-4122. Cleanup HDFS logs and reduce the size of logged messages.
+    (suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -238,6 +241,12 @@ Trunk (Unreleased)
     HDFS-4067. TestUnderReplicatedBlocks intermittently fails due to 
     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
 
     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 
     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
 
   BUG FIXES
@@ -504,6 +516,9 @@ Release 2.0.3-alpha - Unreleased
     out stream returned by WebHdfsFileSystem does not support it. (Jing Zhao
     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 
 
   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-3910. DFSTestUtil#waitReplication should timeout. (eli)
+    
+    HDFS-3920. libwebdhfs string processing and using strerror consistently
+    to handle all errors. (Jing Zhao via suresh)
 
   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_outer_classname = "BKJournalProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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
  * limitations under the License.
  */
+
 #include <stdlib.h>
 #include <string.h>
 #include <curl/curl.h>
-#include <pthread.h>
+
 #include "hdfs_http_client.h"
 
 static pthread_mutex_t curlInitMutex = PTHREAD_MUTEX_INITIALIZER;
 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) {
-        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->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->header);
         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) {
         return 0;
     }
     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) {
-        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;
     }
     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.
  */
-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) {
         return 0;
     }
     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;
     }
     
-    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);
     rbuffer->offset += toCopy;
     rbuffer->remaining -= 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) {
-        fprintf(stderr, "In readfunc callback: size * nmemb == %ld\n", size * nmemb);
         return 0;
     }
-    webhdfsBuffer *wbuffer = (webhdfsBuffer *) stream;
     
+    wbuffer = stream;
     pthread_mutex_lock(&wbuffer->writeMutex);
     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 (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");
             pthread_mutex_unlock(&wbuffer->writeMutex);
             return 0;
         } 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_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);
         wbuffer->offset += copySize;
         wbuffer->remaining -= copySize;
         pthread_mutex_unlock(&wbuffer->writeMutex);
         return copySize;
     } 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);
         return 0;
     }
 }
 
-static void initCurlGlobal() {
+/**
+ * Initialize the global libcurl environment
+ */
+static void initCurlGlobal()
+{
     if (!curlGlobalInited) {
         pthread_mutex_lock(&curlInitMutex);
         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) {
-        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();
-    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);
     }
-    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) {
-        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();
-    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) {
-        fprintf(stderr, "upload buffer is NULL!\n");
-        errno = EINVAL;
-        return NULL;
+        fprintf(stderr, "ERROR: upload buffer is NULL!\n");
+        return EINVAL;
     }
+    
     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) {
-        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);
     
-    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);
+    }
+    if (curl != NULL) {
         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 <unistd.h> /* for size_t */
 
+/** enum indicating the type of hdfs stream */
 enum hdfsStreamType
 {
     UNINITIALIZED = 0,
@@ -36,28 +37,39 @@ enum hdfsStreamType
 /**
  * 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 {
-    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;
 };
 
+/** Type of http header */
 enum HttpHeader {
     GET,
     PUT,
@@ -65,44 +77,218 @@ enum HttpHeader {
     DELETE
 };
 
+/** Whether to redirect */
 enum Redirect {
     YES,
     NO
 };
 
-typedef struct {
+/** Buffer used for holding response */
+struct ResponseBuffer {
     char *content;
     size_t remaining;
     size_t offset;
-} ResponseBufferInternal;
-typedef ResponseBufferInternal *ResponseBuffer;
+};
 
 /**
  * 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_

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

@@ -22,233 +22,381 @@
 #include <stdio.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 {
-        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) {
-        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) {
-        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_
 #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_

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

@@ -25,6 +25,11 @@
 #include <ctype.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
  */
@@ -34,9 +39,6 @@ struct jsonException {
   const char *message;
 };
 
-static hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat,
-                           int *numEntries, const char *operation);
-
 static void dotsToSlashes(char *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;
     int excErrno = EINTERNAL, shouldPrint = 0;
@@ -74,11 +77,23 @@ int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
     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;
-    int ret;
+    int ret = 0;
     
     va_start(ap, fmt);
     ret = printJsonExceptionV(exc, noPrintFlags, fmt, ap);
@@ -86,81 +101,20 @@ int printJsonException(struct jsonException *exc, int noPrintFlags,
     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;
+    void *iter = NULL;
     
     exception = calloc(1, sizeof(*exception));
     if (!exception) {
         return NULL;
     }
     
-    void *iter = json_object_iter(jobj);
+    iter = json_object_iter(jobj);
     while (iter) {
         key = json_object_iter_key(iter);
         value = json_object_iter_value(iter);
@@ -175,23 +129,31 @@ struct jsonException *parseJsonException(json_t *jobj) {
         
         iter = json_object_iter_next(jobj, iter);
     }
-    
     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;
     size_t flags = 0;
-    const char *key;
+    const char *key = NULL;
     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) {
-        fprintf(stderr, "JSon parsing failed\n");
+        fprintf(stderr, "JSon parsing error: on line %d: %s\n",
+                error.line, error.text);
         return NULL;
     }
     void *iter = json_object_iter(jobj);
@@ -199,254 +161,503 @@ struct jsonException *parseException(const char *content) {
         key = json_object_iter_key(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);
     }
-    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 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);
         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)) {
         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");
 }
 
-
-int parseCHOWN(char *header, const char *content) {
+int parseCHOWN(const char *header, const char *content)
+{
     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);
         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) {
-        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);
         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);
         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;
     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_
 #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);
 
+/**
+ * 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);
+
+/**
+ * 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);
+
+/**
+ * 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);
+
+/**
+ * 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);
 
-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_

File diff suppressed because it is too large
+ 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.
       getLeaseRenewer().closeClient(this);
     } catch (IOException ioe) {
-       LOG.info("Exception occurred while aborting the client. " + ioe);
+       LOG.info("Exception occurred while aborting the client " + ioe);
     }
     closeConnectionToNamenode();
   }
@@ -2139,7 +2139,7 @@ public class DFSClient implements java.io.Closeable {
       reportBadBlocks(lblocks);
     } catch (IOException ie) {
       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);
         if(connectFailedOnce) {
           DFSClient.LOG.info("Successfully connected to " + targetAddr +
-                             " for block " + blk.getBlockId());
+                             " for " + blk);
         }
         return chosenNode;
       } catch (IOException ex) {
@@ -736,9 +736,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
         }
         
         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
             + ". Will get new block locations from namenode and retry...");
         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 {
       if (response != null) {
-        DFSClient.LOG.info("Error Recovery for block " + block +
+        DFSClient.LOG.info("Error Recovery for " + block +
         " waiting for responder to exit. ");
         return true;
       }
@@ -1008,7 +1008,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         success = createBlockOutputStream(nodes, 0L, false);
 
         if (!success) {
-          DFSClient.LOG.info("Abandoning block " + block);
+          DFSClient.LOG.info("Abandoning " + block);
           dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName);
           block = null;
           DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
@@ -1773,7 +1773,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         try {
           Thread.sleep(400);
           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) {
           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()}; 
     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="
         + dataNode[0]);
 
@@ -715,7 +715,7 @@ public class DistributedFileSystem extends FileSystem {
     }
     DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
     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]);
 
     // 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(" ");
     }
     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
       // by the Datanode on startup
       NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
-          + blk + " not found.");
+          + blk + " not found");
       return;
     }
     markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn);
@@ -1026,7 +1026,7 @@ public class BlockManager {
       NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " +
           "invalidation of " + b + " on " + dn + " because " +
           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);
 
     } else if (nr.liveReplicas() >= 1) {
@@ -1039,7 +1039,7 @@ public class BlockManager {
       }
     } else {
       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)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
                 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;
               }
             }
@@ -1236,9 +1235,8 @@ public class BlockManager {
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.decrementReplicationIndex(priority);
               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;
             }
           }
@@ -1290,10 +1288,8 @@ public class BlockManager {
             targetList.append(' ');
             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();
       node.receivedBlockReport();
       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();
       }
       
@@ -1601,9 +1596,9 @@ public class BlockManager {
       addStoredBlock(b, node, null, true);
     }
     for (Block b : toInvalidate) {
-      NameNode.stateChangeLog.info("BLOCK* processReport: block "
+      NameNode.stateChangeLog.info("BLOCK* processReport: "
           + b + " on " + node + " size " + b.getNumBytes()
-          + " does not belong to any file.");
+          + " does not belong to any file");
       addToInvalidates(b, node);
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
@@ -1870,7 +1865,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int count = pendingDNMessages.count();
     if (count > 0) {
       LOG.info("Processing " + count + " messages from DataNodes " +
-          "that were previously queued during standby state.");
+          "that were previously queued during standby state");
     }
     processQueuedMessages(pendingDNMessages.takeAll());
     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
           // closed. So, ignore this report, assuming we will get a
           // 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;
         } else {
           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.
       NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
           + 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.
       // it will happen in next block report otherwise.
       return block;
@@ -2158,9 +2153,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       try {
         invalidateBlock(new BlockToMarkCorrupt(blk, null), node);
       } 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;
       }
     }
@@ -2308,7 +2302,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       DatanodeDescriptor cur = it.next();
       if (cur.areBlockContentsStale()) {
         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 information.");
         postponeBlock(block);
@@ -2398,7 +2392,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       //
       addToInvalidates(b, cur);
       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) {
       NameNode.stateChangeLog.info("BLOCK* addBlock: block "
           + b + " on " + node + " size " + b.getNumBytes()
-          + " does not belong to any file.");
+          + " does not belong to any file");
       addToInvalidates(b, node);
     }
     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),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * 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
    * @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) {
     if(recoverBlocks.contains(block)) {
       // 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;
     }
     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 (!blockManager.isReplicationInProgress(node)) {
         node.setDecommissioned();
-        LOG.info("Decommission complete for node " + node);
+        LOG.info("Decommission complete for " + node);
       }
     }
     return node.isDecommissioned();
@@ -593,8 +593,8 @@ public class DatanodeManager {
   /** Start decommissioning the specified datanode. */
   private void startDecommission(DatanodeDescriptor node) {
     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);
       node.decommissioningStatus.setStartTime(now());
       
@@ -606,7 +606,7 @@ public class DatanodeManager {
   /** Stop decommissioning the specified datanodes. */
   void stopDecommission(DatanodeDescriptor node) {
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
-      LOG.info("Stop Decommissioning node " + node);
+      LOG.info("Stop Decommissioning " + node);
       heartbeatManager.stopDecommission(node);
       blockManager.processOverReplicatedBlocksOnReCommission(node);
     }
@@ -658,17 +658,15 @@ public class DatanodeManager {
       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 nodeN = host2DatanodeMap.getDatanodeByXferAddr(
         nodeReg.getIpAddr(), nodeReg.getXferPort());
       
     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, 
       // which is not served by anybody anymore.
       removeDatanode(nodeN);
@@ -683,8 +681,8 @@ public class DatanodeManager {
         // storage. We do not need to remove old data blocks, the delta will
         // be calculated on the next block report from the datanode
         if(NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("BLOCK* NameSystem.registerDatanode: "
-                                        + "node restarted.");
+          NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
+              + "node restarted.");
         }
       } else {
         // nodeS is found
@@ -696,11 +694,9 @@ public class DatanodeManager {
           value in "VERSION" file under the data directory of the datanode,
           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
       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()) {
           // storage directory does not exist
           if (startOpt != StartupOption.FORMAT) {
-            LOG.info("Storage directory " + rootPath + " does not exist.");
+            LOG.info("Storage directory " + rootPath + " does not exist");
             return StorageState.NON_EXISTENT;
           }
           LOG.info(rootPath + " does not exist. Creating ...");
@@ -442,7 +442,7 @@ public abstract class Storage extends StorageInfo {
         }
         // or is inaccessible
         if (!root.isDirectory()) {
-          LOG.info(rootPath + "is not a directory.");
+          LOG.info(rootPath + "is not a directory");
           return StorageState.NON_EXISTENT;
         }
         if (!root.canWrite()) {
@@ -539,34 +539,34 @@ public abstract class Storage extends StorageInfo {
       switch(curState) {
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
         LOG.info("Completing previous upgrade for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         rename(getPreviousTmp(), getPreviousDir());
         return;
       case RECOVER_UPGRADE:   // mv previous.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous upgrade.");
+                 + " from previous upgrade");
         if (curDir.exists())
           deleteDir(curDir);
         rename(getPreviousTmp(), curDir);
         return;
       case COMPLETE_ROLLBACK: // rm removed.tmp
         LOG.info("Completing previous rollback for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getRemovedTmp());
         return;
       case RECOVER_ROLLBACK:  // mv removed.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous rollback.");
+                 + " from previous rollback");
         rename(getRemovedTmp(), curDir);
         return;
       case COMPLETE_FINALIZE: // rm finalized.tmp
         LOG.info("Completing previous finalize for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getFinalizedTmp());
         return;
       case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
         LOG.info("Completing previous checkpoint for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         File prevCkptDir = getPreviousCheckpoint();
         if (prevCkptDir.exists())
           deleteDir(prevCkptDir);
@@ -574,7 +574,7 @@ public abstract class Storage extends StorageInfo {
         return;
       case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
-                 + " from failed checkpoint.");
+                 + " from failed checkpoint");
         if (curDir.exists())
           deleteDir(curDir);
         rename(getLastCheckpointTmp(), curDir);
@@ -629,7 +629,7 @@ public abstract class Storage extends StorageInfo {
       FileLock newLock = tryLock();
       if (newLock == null) {
         String msg = "Cannot lock storage " + this.root 
-          + ". The directory is already locked.";
+          + ". The directory is already locked";
         LOG.info(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;
   
-  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;
 
   Thread bpThread;
   DatanodeProtocolClientSideTranslatorPB bpNamenode;
-  private long lastHeartbeat = 0;
+  private volatile long lastHeartbeat = 0;
   private volatile boolean initialized = false;
   
   /**
@@ -637,8 +641,7 @@ class BPServiceActor implements Runnable {
     try {
       Thread.sleep(millis);
     } 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;
     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
     List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
@@ -310,12 +310,12 @@ class BlockPoolSliceScanner {
   }
   
   private void handleScanFailure(ExtendedBlock block) {
-    LOG.info("Reporting bad block " + block);
+    LOG.info("Reporting bad " + block);
     try {
       datanode.reportBadBlocks(block);
     } catch (IOException ie) {
       // 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 (!dataset.contains(block)) {
-          LOG.info(block + " is no longer in the dataset.");
+          LOG.info(block + " is no longer in the dataset");
           deleteBlock(block.getLocalBlock());
           return;
         }
@@ -424,7 +424,7 @@ class BlockPoolSliceScanner {
         // is a block really deleted by mistake, DirectoryScan should catch it.
         if (e instanceof FileNotFoundException ) {
           LOG.info("Verification failed for " + block +
-              ". It may be due to race with write.");
+              " - may be due to race with write");
           deleteBlock(block.getLocalBlock());
           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
     rename(bpTmpDir, bpPrevDir);
     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
     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 {
     String bpid = block.getBlockPoolId();
     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
       throw ioe;
     } 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);
       if (srcDataNode != null) {
         try {
-          LOG.info("report corrupt block " + block + " from datanode " +
+          LOG.info("report corrupt " + block + " from datanode " +
                     srcDataNode + " to namenode");
           datanode.reportRemoteBadBlock(srcDataNode, block);
         } catch (IOException e) {
-          LOG.warn("Failed to report bad block " + block + 
+          LOG.warn("Failed to report bad " + block + 
                     " 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 (firstByteInBlock % bytesPerChecksum != 0) {
             LOG.info("Packet starts at " + firstByteInBlock +
-                     " for block " + block +
+                     " for " + block +
                      " which is not a multiple of bytesPerChecksum " +
                      bytesPerChecksum);
             long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
@@ -662,7 +662,7 @@ class BlockReceiver implements Closeable {
       }
 
     } catch (IOException ioe) {
-      LOG.info("Exception in receiveBlock for " + block, ioe);
+      LOG.info("Exception for " + block, ioe);
       throw ioe;
     } finally {
       if (!responderClosed) { // Abnormal termination of the flow above
@@ -733,10 +733,9 @@ class BlockReceiver implements Closeable {
     int checksumSize = diskChecksum.getChecksumSize();
     blkoff = blkoff - 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
     // and read in partial crc chunk into temporary buffer
@@ -758,7 +757,7 @@ class BlockReceiver implements Closeable {
     partialCrc = DataChecksum.newDataChecksum(
         diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum());
     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
     // recalculated just now
@@ -973,7 +972,7 @@ class BlockReceiver implements Closeable {
                       "HDFS_WRITE", clientname, offset,
                       dnR.getStorageID(), block, endTime-startTime));
               } else {
-                LOG.info("Received block " + block + " of size "
+                LOG.info("Received " + block + " size "
                     + 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
          * the socket open).
          */
-          LOG.info("BlockSender.sendChunks() exception: ", e);
+          LOG.info("exception: ", e);
       } else {
         /* Exception while writing to the client. Connection closure from
          * 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.start();
     } 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();
     } else {
       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;
     shutdownPeriodicScanners();
     
@@ -1141,7 +1146,7 @@ public class DataNode extends Configured
     
     if(blockPoolManager != null) {
       try {
-        this.blockPoolManager.shutDownAll();
+        this.blockPoolManager.shutDownAll(bposArray);
       } catch (InterruptedException ie) {
         LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie);
       }
@@ -1256,7 +1261,7 @@ public class DataNode extends Configured
           xfersBuilder.append(xferTargets[i]);
           xfersBuilder.append(" ");
         }
-        LOG.info(bpReg + " Starting thread to transfer block " + 
+        LOG.info(bpReg + " Starting thread to transfer " + 
                  block + " to " + xfersBuilder);                       
       }
 
@@ -2043,7 +2048,7 @@ public class DataNode extends Configured
     ExtendedBlock block = rb.getBlock();
     DatanodeInfo[] targets = rb.getLocations();
     
-    LOG.info(who + " calls recoverBlock(block=" + block
+    LOG.info(who + " calls recoverBlock(" + block
         + ", targets=[" + Joiner.on(", ").join(targets) + "]"
         + ", 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;
         case NON_EXISTENT:
           // ignore this storage
-          LOG.info("Storage directory " + dataDir + " does not exist.");
+          LOG.info("Storage directory " + dataDir + " does not exist");
           it.remove();
           continue;
         case NOT_FORMATTED: // format
-          LOG.info("Storage directory " + dataDir + " is not formatted.");
+          LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Formatting ...");
           format(sd, nsInfo);
           break;
@@ -482,7 +482,7 @@ public class DataStorage extends Storage {
     
     // 5. Rename <SD>/previous.tmp to <SD>/previous
     rename(tmpDir, prevDir);
-    LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
+    LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
     addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage);
   }
 
@@ -556,7 +556,7 @@ public class DataStorage extends Storage {
     rename(prevDir, curDir);
     // delete tmp dir
     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);
             }
           } 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
         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) {
           LOG.info("Failed to read expected encryption handshake from client " +
               "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;
         }
         input = encryptedStreams.in;
@@ -367,9 +367,8 @@ class DataXceiver extends Receiver implements Runnable {
     // make a copy here.
     final ExtendedBlock originalBlock = new ExtendedBlock(block);
     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 
     final DataOutputStream replyOut = new DataOutputStream(
@@ -478,9 +477,9 @@ class DataXceiver extends Receiver implements Runnable {
                       block + " to mirror " + mirrorNode + ": " + e);
             throw e;
           } else {
-            LOG.info(datanode + ":Exception transfering block " +
+            LOG.info(datanode + ":Exception transfering " +
                      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 ||
           stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
         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.incrBlocksRead();
       
-      LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
+      LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress());
     } catch (IOException ioe) {
       isOpSuccess = false;
       LOG.info("opCopyBlock " + block + " received exception " + ioe);
@@ -797,8 +794,7 @@ class DataXceiver extends Receiver implements Runnable {
       // notify name node
       datanode.notifyNamenodeReceivedBlock(block, delHint);
 
-      LOG.info("Moved block " + block + 
-          " from " + s.getRemoteSocketAddress());
+      LOG.info("Moved " + block + " from " + s.getRemoteSocketAddress());
       
     } catch (IOException ioe) {
       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) {
       LOG.warn("AsyncDiskService has already shut down.");
     } 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()) {
         e.getValue().shutdown();
@@ -144,7 +144,7 @@ class FsDatasetAsyncDiskService {
       // clear the executor map so that calling execute again will fail.
       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,
       ExtendedBlock block) {
-    LOG.info("Scheduling block " + block.getLocalBlock()
+    LOG.info("Scheduling " + block.getLocalBlock()
         + " file " + blockFile + " for deletion");
     ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask(
         volume, blockFile, metaFile, block);
@@ -198,8 +198,8 @@ class FsDatasetAsyncDiskService {
           datanode.notifyNamenodeDeletedBlock(block);
         }
         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;
     }
     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 + ")");
     }
 
@@ -481,7 +481,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           " should be greater than the replica " + b + "'s generation stamp");
     }
     ReplicaInfo replicaInfo = getReplicaInfo(b);
-    LOG.info("Appending to replica " + replicaInfo);
+    LOG.info("Appending to " + replicaInfo);
     if (replicaInfo.getState() != ReplicaState.FINALIZED) {
       throw new ReplicaNotFoundException(
           ReplicaNotFoundException.UNFINALIZED_REPLICA + b);
@@ -689,7 +689,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b,
       long newGS, long minBytesRcvd, long maxBytesRcvd)
       throws IOException {
-    LOG.info("Recover the RBW replica " + b);
+    LOG.info("Recover RBW replica " + b);
 
     ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId());
     
@@ -700,7 +700,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
     ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo;
     
-    LOG.info("Recovering replica " + rbw);
+    LOG.info("Recovering " + rbw);
 
     // Stop the previous writer
     rbw.stopWriter();
@@ -736,8 +736,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final long blockId = b.getBlockId();
     final long expectedGs = b.getGenerationStamp();
     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;
     {
@@ -1415,8 +1415,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   static ReplicaRecoveryInfo initReplicaRecovery(String bpid,
       ReplicaMap map, Block block, long recoveryId) throws IOException {
     final ReplicaInfo replica = map.get(bpid, block.getBlockId());
-    LOG.info("initReplicaRecovery: block=" + block
-        + ", recoveryId=" + recoveryId
+    LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId
         + ", replica=" + replica);
 
     //check replica
@@ -1485,7 +1484,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     //get replica
     final String bpid = oldBlock.getBlockPoolId();
     final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId());
-    LOG.info("updateReplica: block=" + oldBlock
+    LOG.info("updateReplica: " + oldBlock
         + ", recoveryId=" + recoveryId
         + ", length=" + newlength
         + ", 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_DEFAULT);
     NameNode.LOG.info("Caching file names occuring more than " + threshold
-        + " times ");
+        + " times");
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;
   }
@@ -255,15 +255,12 @@ public class FSDirectory implements Closeable {
       writeUnlock();
     }
     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;
     }
 
     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;
   }
@@ -2189,16 +2186,13 @@ public class FSDirectory implements Closeable {
       writeUnlock();
     }
     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;
     }
     fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode);
     
     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;
   }

+ 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.
    */
   synchronized long rollEditLog() throws IOException {
-    LOG.info("Rolling edit logs.");
+    LOG.info("Rolling edit logs");
     endCurrentLogSegment(true);
     
     long nextTxId = getLastWrittenTxId() + 1;
@@ -943,7 +943,7 @@ public class FSEditLog implements LogsPurgeable {
    */
   public synchronized void startLogSegment(long txid, 
       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 (getLastWrittenTxId() == txid - 1) {
         //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);
       FSImage.LOG.info("Edits file " + edits.getName() 
           + " of size " + edits.length() + " edits # " + numEdits 
-          + " loaded in " + (now()-startTime)/1000 + " seconds.");
+          + " loaded in " + (now()-startTime)/1000 + " seconds");
       return numEdits;
     } finally {
       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();
         
         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();
         blockManager.setPostponeBlocksFromFuture(false);
         
-        LOG.info("Reprocessing replication and invalidation queues...");
+        LOG.info("Reprocessing replication and invalidation queues");
         blockManager.getDatanodeManager().markAllDatanodesStale();
         blockManager.clearQueues();
         blockManager.processAllPendingDNMessages();
@@ -1978,7 +1978,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (force) {
         // close now: no need to wait for soft lease expiration and 
         // close only the file src
-        LOG.info("recoverLease: recover lease " + lease + ", src=" + src +
+        LOG.info("recoverLease: " + lease + ", src=" + src +
           " from client " + pendingFile.getClientName());
         internalReleaseLease(lease, src, holder);
       } else {
@@ -1990,8 +1990,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // period, then start lease recovery.
         //
         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);
           if(!isClosed)
             throw new RecoveryInProgressException(
@@ -2167,7 +2167,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           }
 
           // 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 " +
               src + ". Abandoning old block " + lastBlockInFile);
           dir.removeBlock(src, pendingFile, lastBlockInFile);
@@ -2403,10 +2403,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // See HDFS-3031.
         final Block realLastBlock = ((INodeFile)inode).getLastBlock();
         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 " +
-              "retry. Returning success.");
+              "retry. Returning success");
           return true;
         }
       }
@@ -2421,8 +2421,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     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;
   }
 
@@ -2447,8 +2447,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     nextGenerationStamp();
     b.setGenerationStamp(getGenerationStamp());
     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;
   }
 
@@ -2466,8 +2466,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         //
         for (BlockInfo block: v.getBlocks()) {
           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);
             return false;
           }
@@ -2478,8 +2478,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         //
         BlockInfo b = v.getPenultimateBlock();
         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);
           return false;
         }
@@ -2952,8 +2952,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void fsync(String src, String clientName) 
       throws IOException, UnresolvedLinkException {
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
-                                  + src + " for " + clientName);
+    NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2984,7 +2983,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   boolean internalReleaseLease(Lease lease, String src, 
       String recoveryLeaseHolder) throws AlreadyBeingCreatedException, 
       IOException, UnresolvedLinkException {
-    LOG.info("Recovering lease=" + lease + ", src=" + src);
+    LOG.info("Recovering " + lease + ", src=" + src);
     assert !isInSafeMode();
     assert hasWriteLock();
 
@@ -3625,7 +3624,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                               "in order to create namespace image.");
       }
       getFSImage().saveNamespace(this);
-      LOG.info("New namespace image has been created.");
+      LOG.info("New namespace image has been created");
     } finally {
       readUnlock();
     }
@@ -3843,11 +3842,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
       long timeInSafemode = now() - startTime;
       NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
-                                    + timeInSafemode/1000 + " secs.");
+                                    + timeInSafemode/1000 + " secs");
       NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
       
       if (reached >= 0) {
-        NameNode.stateChangeLog.info("STATE* Safe mode is OFF."); 
+        NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 
       }
       reached = -1;
       safeMode = null;
@@ -4167,7 +4166,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         }
       }
       if (!fsRunning) {
-        LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread. ");
+        LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
       } else {
         // leave safe mode and stop the monitor
         leaveSafeMode();
@@ -4352,7 +4351,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isEditlogOpenForWrite) {
         getEditLog().logSyncAll();
       }
-      NameNode.stateChangeLog.info("STATE* Safe mode is ON. "
+      NameNode.stateChangeLog.info("STATE* Safe mode is ON"
           + safeMode.getTurnOffTip());
     } finally {
       writeUnlock();
@@ -4367,7 +4366,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       if (!isInSafeMode()) {
-        NameNode.stateChangeLog.info("STATE* Safe mode is already OFF."); 
+        NameNode.stateChangeLog.info("STATE* Safe mode is already OFF"); 
         return;
       }
       safeMode.leave();
@@ -4721,7 +4720,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       checkOperation(OperationCategory.WRITE);
       
-      NameNode.stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
+      NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
       for (int i = 0; i < blocks.length; i++) {
         ExtendedBlock blk = blocks[i].getBlock();
         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) {
-      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);
     } finally {
       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;
       }
 
-      LOG.info("Lease " + oldest + " has expired hard limit");
+      LOG.info(oldest + " has expired hard limit");
 
       final List<String> removing = new ArrayList<String>();
       // need to create a copy of the oldest lease paths, becuase 
@@ -441,15 +441,14 @@ public class LeaseManager {
       for(String p : leasePaths) {
         try {
           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);
           } else {
-            LOG.info("Started block recovery for file " + p +
-                          " lease " + oldest);
+            LOG.info("Started block recovery " + p + " lease " + oldest);
           }
         } 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);
         }
       }

+ 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. " + 
       "(c/s/q/a)\n", "c", "s", "q", "a");
     if (answer.equals("c")) {
-      LOG.info("Continuing.");
+      LOG.info("Continuing");
       return;
     } else if (answer.equals("s")) {
       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) {
                 fprintf(stderr, "error creating hashtable, <%d>: %s\n",
                         errno, strerror(errno));
+                UNLOCK_HASH_TABLE();
                 return 0;
             } 
             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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 /**
  *  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_outer_classname = "HAZKInfoProtos";
+package hadoop.hdfs;
 
 message ActiveNodeInfo {
   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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 /**
  *  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_generic_services = true;
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 /**
  *  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_outer_classname = "DataTransferProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 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_outer_classname = "HdfsProtos";
 option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
 
 /**
  * Extended block idenfies a block

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

@@ -15797,7 +15797,7 @@
       <comparators>
         <comparator>
           <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>
           <type>RegexpComparator</type>
@@ -15915,7 +15915,7 @@
       <comparators>
         <comparator>
           <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>
           <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
     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
 
   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(
         TaskStateInternal.SUCCEEDED, TaskStateInternal.SUCCEEDED,
         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        
     .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());    
   }
 
+  @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 
   public void testTaskProgress() {
     LOG.info("--- START: testTaskProgress ---");
@@ -485,7 +500,8 @@ public class TestTaskImpl {
     assertTaskSucceededState();
   }
   
-  private void runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType failEvent) {
+  private void runSpeculativeTaskAttemptSucceeds(
+      TaskEventType firstAttemptFinishEvent) {
     TaskId taskId = getNewTaskID();
     scheduleTaskAttempt(taskId);
     launchTaskAttempt(getLastAttempt().getAttemptId());
@@ -502,9 +518,9 @@ public class TestTaskImpl {
     // The task should now have succeeded
     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(), 
-        failEvent));
+        firstAttemptFinishEvent));
     
     // The task should still be in the succeeded state
     assertTaskSucceededState();
@@ -513,25 +529,36 @@ public class TestTaskImpl {
   @Test
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
     mockTask = createMockTask(TaskType.MAP);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_FAILED);
   }
 
   @Test
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstFails() {
     mockTask = createMockTask(TaskType.REDUCE);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_FAILED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_FAILED);
   }
   
   @Test
   public void testMapSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
     mockTask = createMockTask(TaskType.MAP);        
-    runSpeculativeTaskAttemptSucceedsEvenIfFirstFails(TaskEventType.T_ATTEMPT_KILLED);
+    runSpeculativeTaskAttemptSucceeds(TaskEventType.T_ATTEMPT_KILLED);
   }
 
   @Test
   public void testReduceSpeculativeTaskAttemptSucceedsEvenIfFirstIsKilled() {
     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
 
+    YARN-145. Add a Web UI to the fair share scheduler. (Sandy Ryza via tomwhite)
+
   IMPROVEMENTS
 
     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) {
     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) {

+ 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);
 
   private final AppSchedulingInfo appSchedulingInfo;
+  private AppSchedulable appSchedulable;
   private final Queue queue;
 
   private final Resource currentConsumption = recordFactory
@@ -118,6 +119,14 @@ public class FSSchedulerApp extends SchedulerApplication {
   public ApplicationAttemptId getApplicationAttemptId() {
     return this.appSchedulingInfo.getApplicationAttemptId();
   }
+  
+  public void setAppSchedulable(AppSchedulable appSchedulable) {
+    this.appSchedulable = appSchedulable;
+  }
+  
+  public AppSchedulable getAppSchedulable() {
+    return appSchedulable;
+  }
 
   public String 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);
     return node == null ? null : new SchedulerNodeReport(node);
   }
-
+  
+  public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
+    return applications.get(appAttemptId);
+  }
+  
   @Override
   public SchedulerAppReport getSchedulerAppInfo(
       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.
    * @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) {
       if (maxQueueResources.containsKey(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) {
-      context().setStatus(404);
-      throw new WebAppException("Fair Scheduler UI not yet supported");
+      setTitle("Fair Scheduler");
+      render(FairSchedulerPage.class);
+      return;
     }
     
     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;
+  }
+}

Some files were not shown because too many files changed in this diff