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
This commit is contained in:
Suresh Srinivas 2012-10-29 14:09:47 +00:00
commit 564adec5b5
73 changed files with 3099 additions and 1278 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -25,6 +25,11 @@
#include <ctype.h> #include <ctype.h>
#include <jansson.h> #include <jansson.h>
static const char * const temporaryRedirectCode = "307 TEMPORARY_REDIRECT";
static const char * const twoHundredOKCode = "200 OK";
static const char * const twoHundredOneCreatedCode = "201 Created";
static const char * const httpHeaderString = "HTTP/1.1";
/** /**
* Exception information after calling JSON operations * Exception information after calling JSON operations
*/ */
@ -34,9 +39,6 @@ struct jsonException {
const char *message; const char *message;
}; };
static hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat,
int *numEntries, const char *operation);
static void dotsToSlashes(char *str) static void dotsToSlashes(char *str)
{ {
for (; *str != '\0'; str++) { for (; *str != '\0'; str++) {
@ -45,8 +47,9 @@ static void dotsToSlashes(char *str)
} }
} }
int printJsonExceptionV(struct jsonException *exc, int noPrintFlags, /** Print out the JSON exception information */
const char *fmt, va_list ap) static int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
const char *fmt, va_list ap)
{ {
char *javaClassName = NULL; char *javaClassName = NULL;
int excErrno = EINTERNAL, shouldPrint = 0; int excErrno = EINTERNAL, shouldPrint = 0;
@ -74,11 +77,23 @@ int printJsonExceptionV(struct jsonException *exc, int noPrintFlags,
return excErrno; return excErrno;
} }
int printJsonException(struct jsonException *exc, int noPrintFlags, /**
const char *fmt, ...) * Print out JSON exception information.
*
* @param exc The exception information to print and free
* @param noPrintFlags Flags which determine which exceptions we should NOT
* print.
* @param fmt Printf-style format list
* @param ... Printf-style varargs
*
* @return The POSIX error number associated with the exception
* object.
*/
static int printJsonException(struct jsonException *exc, int noPrintFlags,
const char *fmt, ...)
{ {
va_list ap; va_list ap;
int ret; int ret = 0;
va_start(ap, fmt); va_start(ap, fmt);
ret = printJsonExceptionV(exc, noPrintFlags, fmt, ap); ret = printJsonExceptionV(exc, noPrintFlags, fmt, ap);
@ -86,81 +101,20 @@ int printJsonException(struct jsonException *exc, int noPrintFlags,
return ret; return ret;
} }
static hdfsFileInfo *json_parse_array(json_t *jobj, char *key, hdfsFileInfo *fileStat, int *numEntries, const char *operation) { /** Parse the exception information from JSON */
int arraylen = json_array_size(jobj); //Getting the length of the array static struct jsonException *parseJsonException(json_t *jobj)
*numEntries = arraylen; {
if (!key) { const char *key = NULL;
return NULL; json_t *value = 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;
struct jsonException *exception = NULL; struct jsonException *exception = NULL;
void *iter = NULL;
exception = calloc(1, sizeof(*exception)); exception = calloc(1, sizeof(*exception));
if (!exception) { if (!exception) {
return NULL; return NULL;
} }
void *iter = json_object_iter(jobj); iter = json_object_iter(jobj);
while (iter) { while (iter) {
key = json_object_iter_key(iter); key = json_object_iter_key(iter);
value = json_object_iter_value(iter); value = json_object_iter_value(iter);
@ -175,23 +129,31 @@ struct jsonException *parseJsonException(json_t *jobj) {
iter = json_object_iter_next(jobj, iter); iter = json_object_iter_next(jobj, iter);
} }
return exception; return exception;
} }
struct jsonException *parseException(const char *content) { /**
* 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 = NULL;
json_t *value;
json_t *jobj;
struct jsonException *exception = NULL;
if (!content) { if (!content) {
return NULL; return NULL;
} }
jobj = json_loads(content, flags, &error);
json_error_t error;
size_t flags = 0;
const char *key;
json_t *value;
json_t *jobj = json_loads(content, flags, &error);
if (!jobj) { if (!jobj) {
fprintf(stderr, "JSon parsing failed\n"); fprintf(stderr, "JSon parsing error: on line %d: %s\n",
error.line, error.text);
return NULL; return NULL;
} }
void *iter = json_object_iter(jobj); void *iter = json_object_iter(jobj);
@ -199,254 +161,503 @@ struct jsonException *parseException(const char *content) {
key = json_object_iter_key(iter); key = json_object_iter_key(iter);
value = json_object_iter_value(iter); value = json_object_iter_value(iter);
if (!strcmp(key, "RemoteException") && json_typeof(value) == JSON_OBJECT) { if (!strcmp(key, "RemoteException") &&
return parseJsonException(value); json_typeof(value) == JSON_OBJECT) {
exception = parseJsonException(value);
break;
} }
iter = json_object_iter_next(jobj, iter); iter = json_object_iter_next(jobj, iter);
} }
return NULL;
json_decref(jobj);
return exception;
} }
static hdfsFileInfo *parseJsonGFS(json_t *jobj, hdfsFileInfo *fileStat, /**
int *numEntries, const char *operation) * Parse the response information which uses TRUE/FALSE
* to indicate whether the operation succeeded
*
* @param response Response information
* @return 0 to indicate success
*/
static int parseBoolean(const char *response)
{ {
const char *tempstr; json_t *root, *value;
const char *key; json_error_t error;
json_t *value; size_t flags = 0;
void *iter = json_object_iter(jobj); int result = 0;
while(iter) {
key = json_object_iter_key(iter); root = json_loads(response, flags, &error);
value = json_object_iter_value(iter); if (!root) {
fprintf(stderr, "JSon parsing error: on line %d: %s\n",
switch (json_typeof(value)) { error.line, error.text);
case JSON_INTEGER: return EIO;
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);
} }
return fileStat; 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
}
json_decref(root);
return result;
} }
int parseMKDIR(const char *response)
{
return parseBoolean(response);
}
int checkHeader(char *header, const char *content, const char *operation) { int parseRENAME(const char *response)
{
return parseBoolean(response);
}
int parseDELETE(const char *response)
{
return parseBoolean(response);
}
int parseSETREPLICATION(const char *response)
{
return parseBoolean(response);
}
/**
* Check the header of response to see if it's 200 OK
*
* @param header Header information for checking
* @param content Stores exception information if there are errors
* @param operation Indicate the operation for exception printing
* @return 0 for success
*/
static int checkHeader(const char *header, const char *content,
const char *operation)
{
char *result = NULL; char *result = NULL;
char delims[] = ":"; const char delims[] = ":";
char *responseCode= "200 OK"; char *savepter;
if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) { int ret = 0;
return 0;
if (!header || strncmp(header, "HTTP/", strlen("HTTP/"))) {
return EINVAL;
} }
if(!(strstr(header, responseCode)) || !(header = strstr(header, "Content-Length"))) { if (!(strstr(header, twoHundredOKCode)) ||
!(result = strstr(header, "Content-Length"))) {
struct jsonException *exc = parseException(content); struct jsonException *exc = parseException(content);
if (exc) { if (exc) {
errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation); ret = printJsonException(exc, PRINT_EXC_ALL,
"Calling WEBHDFS (%s)", operation);
} else {
ret = EIO;
} }
return 0; return ret;
} }
result = strtok(header, delims); result = strtok_r(result, delims, &savepter);
result = strtok(NULL,delims); result = strtok_r(NULL, delims, &savepter);
while (isspace(*result)) { while (isspace(*result)) {
result++; result++;
} }
if(strcmp(result,"0")) { //Content-Length should be equal to 0 // Content-Length should be equal to 0,
return 1; // and the string should be "0\r\nServer"
} else { if (strncmp(result, "0\r\n", 3)) {
return 0; ret = EIO;
} }
return ret;
} }
int parseOPEN(const char *header, const char *content) { int parseCHMOD(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) {
return checkHeader(header, content, "CHMOD"); return checkHeader(header, content, "CHMOD");
} }
int parseCHOWN(const char *header, const char *content)
int parseCHOWN(char *header, const char *content) { {
return checkHeader(header, content, "CHOWN"); return checkHeader(header, content, "CHOWN");
} }
int parseUTIMES(char *header, const char *content) { int parseUTIMES(const char *header, const char *content)
return checkHeader(header, content, "UTIMES"); {
return checkHeader(header, content, "SETTIMES");
} }
/**
int checkIfRedirect(const char *const headerstr, const char *content, const char *operation) { * Check if the header contains correct information
char *responseCode = "307 TEMPORARY_REDIRECT"; * ("307 TEMPORARY_REDIRECT" and "Location")
char * locTag = "Location"; *
char * tempHeader; * @param header Header for parsing
if(headerstr == '\0' || strncmp(headerstr,"HTTP/", 5)) { * @param content Contains exception information
return 0; * 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;
} }
if(!(tempHeader = strstr(headerstr,responseCode))) {
//process possible exception information offset += strlen(httpHeaderString);
while (isspace(header[offset])) {
offset++;
}
// Looking for "307 TEMPORARY_REDIRECT" in header
if (strncmp(header + offset, temporaryRedirectCode,
strlen(temporaryRedirectCode))) {
// Process possible exception information
struct jsonException *exc = parseException(content); struct jsonException *exc = parseException(content);
if (exc) { if (exc) {
errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (%s)", operation); ret = printJsonException(exc, PRINT_EXC_ALL,
"Calling WEBHDFS (%s)", operation);
} else {
ret = EIO;
} }
return 0; return ret;
} }
if(!(strstr(tempHeader,locTag))) { // Here we just simply check if header contains "Location" tag,
return 0; // detailed processing is in parseDnLoc
if (!(strstr(header, locTag))) {
ret = EIO;
} }
return 1; return ret;
} }
int parseNnWRITE(const char *header, const char *content)
int parseNnWRITE(const char *header, const char *content) { {
return checkIfRedirect(header, content, "Write(NameNode)"); return checkRedirect(header, content, "Write(NameNode)");
} }
int parseNnAPPEND(const char *header, const char *content)
int parseNnAPPEND(const char *header, const char *content) { {
return checkIfRedirect(header, content, "Append(NameNode)"); return checkRedirect(header, content, "Append(NameNode)");
} }
char *parseDnLoc(char *content) { /** 0 for success , -1 for out of range, other values for error */
char delims[] = "\r\n"; int parseOPEN(const char *header, const char *content)
char *url = NULL; {
char *DnLocation = NULL; int ret = 0, offset = 0;
char *savepter;
DnLocation = strtok_r(content, delims, &savepter); if (!header || strncmp(header, httpHeaderString,
while (DnLocation && strncmp(DnLocation, "Location:", strlen("Location:"))) { strlen(httpHeaderString))) {
DnLocation = strtok_r(NULL, delims, &savepter); return EINVAL;
} }
if (!DnLocation) {
return NULL; offset += strlen(httpHeaderString);
while (isspace(header[offset])) {
offset++;
} }
DnLocation = strstr(DnLocation, "http"); if (strncmp(header + offset, temporaryRedirectCode,
if (!DnLocation) { strlen(temporaryRedirectCode)) ||
return NULL; !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;
}
} }
url = malloc(strlen(DnLocation) + 1); return ret;
}
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;
}
dnLocation = strtok_r(tempContent, delims, &savepter);
while (dnLocation && strncmp(dnLocation, "Location:",
strlen("Location:"))) {
dnLocation = strtok_r(NULL, delims, &savepter);
}
if (!dnLocation) {
return EIO;
}
while (isspace(*dnLocation)) {
dnLocation++;
}
if (strncmp(dnLocation, prefix, strlen(prefix))) {
return EIO;
}
url = strdup(dnLocation + strlen(prefixToRemove));
if (!url) { if (!url) {
return NULL; return ENOMEM;
} }
strcpy(url, DnLocation); *dn = url;
return url; return 0;
} }
int parseDnWRITE(const char *header, const char *content) { int parseDnWRITE(const char *header, const char *content)
char *responseCode = "201 Created"; {
fprintf(stderr, "\nheaderstr is: %s\n", header); int ret = 0;
if(header == '\0' || strncmp(header,"HTTP/",strlen("HTTP/"))) { if (header == NULL || header[0] == '\0' ||
return 0; strncmp(header, "HTTP/", strlen("HTTP/"))) {
return EINVAL;
} }
if(!(strstr(header,responseCode))) { if (!(strstr(header, twoHundredOneCreatedCode))) {
struct jsonException *exc = parseException(content); struct jsonException *exc = parseException(content);
if (exc) { if (exc) {
errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (WRITE(DataNode))"); ret = printJsonException(exc, PRINT_EXC_ALL,
"Calling WEBHDFS (WRITE(DataNode))");
} else {
ret = EIO;
} }
return 0;
} }
return 1; return ret;
} }
int parseDnAPPEND(const char *header, const char *content) { int parseDnAPPEND(const char *header, const char *content)
char *responseCode = "200 OK"; {
if(header == '\0' || strncmp(header, "HTTP/", strlen("HTTP/"))) { int ret = 0;
return 0;
if (header == NULL || header[0] == '\0' ||
strncmp(header, "HTTP/", strlen("HTTP/"))) {
return EINVAL;
} }
if(!(strstr(header, responseCode))) { if (!(strstr(header, twoHundredOKCode))) {
struct jsonException *exc = parseException(content); struct jsonException *exc = parseException(content);
if (exc) { if (exc) {
errno = printJsonException(exc, PRINT_EXC_ALL, "Calling WEBHDFS (APPEND(DataNode))"); ret = printJsonException(exc, PRINT_EXC_ALL,
"Calling WEBHDFS (APPEND(DataNode))");
} else {
ret = EIO;
} }
return 0;
} }
return 1; return ret;
} }
hdfsFileInfo *parseGFS(char *str, hdfsFileInfo *fileStat, int *numEntries) { /**
* 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;
}
}
// Go to the next key-value pair in the json object
iter = json_object_iter_next(jobj, iter);
}
return 0;
}
int parseGFS(const char *response, hdfsFileInfo *fileStat, int printError)
{
int ret = 0, printFlag;
json_error_t error; json_error_t error;
size_t flags = 0; size_t flags = 0;
json_t *jobj = json_loads(str, flags, &error); json_t *jobj, *value;
fileStat = parseJsonGFS(jobj, fileStat, numEntries, "GETPATHSTATUS/LISTSTATUS"); const char *key;
return fileStat; 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;
}

View File

@ -18,41 +18,161 @@
#ifndef _HDFS_JSON_PARSER_H_ #ifndef _HDFS_JSON_PARSER_H_
#define _HDFS_JSON_PARSER_H_ #define _HDFS_JSON_PARSER_H_
struct jsonException; /**
* Parse the response for MKDIR request. The response uses TRUE/FALSE
* to indicate whether the operation succeeded.
*
* @param response The response information to parse.
* @return 0 for success
*/
int parseMKDIR(const char *response);
/** /**
* Print out JSON exception information. * Parse the response for RENAME request. The response uses TRUE/FALSE
* to indicate whether the operation succeeded.
* *
* @param exc The exception information to print and free * @param response The response information to parse.
* @param noPrintFlags Flags which determine which exceptions we should NOT * @return 0 for success
* print.
* @param fmt Printf-style format list
* @param ... Printf-style varargs
*
* @return The POSIX error number associated with the exception
* object.
*/ */
int printJsonException(struct jsonException *exc, int noPrintFlags, int parseRENAME(const char *response);
const char *fmt, ...);
int parseMKDIR(char *response); /**
int parseRENAME(char *response); * Parse the response for DELETE request. The response uses TRUE/FALSE
int parseDELETE (char *response); * to indicate whether the operation succeeded.
int parseSETREPLICATION(char *response); *
* @param response The response information to parse.
* @return 0 for success
*/
int parseDELETE(const char *response);
/**
* Parse the response for SETREPLICATION request. The response uses TRUE/FALSE
* to indicate whether the operation succeeded.
*
* @param response The response information to parse.
* @return 0 for success
*/
int parseSETREPLICATION(const char *response);
/**
* Parse the response for OPEN (read) request. A successful operation
* will return "200 OK".
*
* @param response The response information for parsing
* @return 0 for success , -1 for out of range, other values for error
*/
int parseOPEN(const char *header, const char *content); int parseOPEN(const char *header, const char *content);
/**
* Parse the response for WRITE (from NameNode) request.
* A successful operation should return "307 TEMPORARY_REDIRECT" in its header.
*
* @param header The header of the http response
* @param content If failing, the exception message
* sent from NameNode is stored in content
* @return 0 for success
*/
int parseNnWRITE(const char *header, const char *content); int parseNnWRITE(const char *header, const char *content);
/**
* Parse the response for WRITE (from DataNode) request.
* A successful operation should return "201 Created" in its header.
*
* @param header The header of the http response
* @param content If failing, the exception message
* sent from DataNode is stored in content
* @return 0 for success
*/
int parseDnWRITE(const char *header, const char *content); int parseDnWRITE(const char *header, const char *content);
/**
* Parse the response for APPEND (sent from NameNode) request.
* A successful operation should return "307 TEMPORARY_REDIRECT" in its header.
*
* @param header The header of the http response
* @param content If failing, the exception message
* sent from NameNode is stored in content
* @return 0 for success
*/
int parseNnAPPEND(const char *header, const char *content); int parseNnAPPEND(const char *header, const char *content);
/**
* Parse the response for APPEND (from DataNode) request.
* A successful operation should return "200 OK" in its header.
*
* @param header The header of the http response
* @param content If failing, the exception message
* sent from DataNode is stored in content
* @return 0 for success
*/
int parseDnAPPEND(const char *header, const char *content); int parseDnAPPEND(const char *header, const char *content);
char* parseDnLoc(char *content); /**
* Parse the response (from NameNode) to get the location information
* of the DataNode that should be contacted for the following write operation.
*
* @param content Content of the http header
* @param dn To store the location of the DataNode for writing
* @return 0 for success
*/
int parseDnLoc(char *content, char **dn) __attribute__ ((warn_unused_result));
hdfsFileInfo *parseGFS(char *response, hdfsFileInfo *fileStat, int *numEntries); /**
* 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);
int parseCHOWN (char *header, const char *content); /**
int parseCHMOD (char *header, const char *content); * Parse the response for LISTSTATUS operation.
int parseUTIMES(char *header, const char *content); *
* @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);
#endif //_FUSE_JSON_PARSER_H /**
* 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 //_HDFS_JSON_PARSER_H_

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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");
}
}
}

View File

@ -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);
}
}

View File

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

View File

@ -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();
}
}

View File

@ -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;
}
}