You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by wl...@apache.org on 2016/07/06 01:45:12 UTC

[5/5] incubator-hawq git commit: HAWQ-891. Refine libyarn codes

HAWQ-891. Refine libyarn codes


Project: http://git-wip-us.apache.org/repos/asf/incubator-hawq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hawq/commit/383818c2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hawq/tree/383818c2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hawq/diff/383818c2

Branch: refs/heads/master
Commit: 383818c280f3b4787c726920105585c09bd7630f
Parents: 8b79e10
Author: Wen Lin <wl...@pivotal.io>
Authored: Wed Jul 6 09:33:56 2016 +0800
Committer: Wen Lin <wl...@pivotal.io>
Committed: Wed Jul 6 09:33:56 2016 +0800

----------------------------------------------------------------------
 depends/libyarn/mock/MockApplicationClient.h    |    7 +-
 .../MockApplicationClientProtocolInternal.h     |   51 +
 .../MockApplicationMasterProtocolInternal.h     |   51 +
 .../MockContainerManagementProtocolInternal.h   |   52 +
 depends/libyarn/src/CMakeLists.txt              |    2 +-
 depends/libyarn/src/common/Exception.cpp        |   26 -
 depends/libyarn/src/common/Exception.h          |  159 ---
 .../src/libyarnclient/ApplicationClient.cpp     |    6 +-
 .../src/libyarnclient/ApplicationClient.h       |   10 +-
 .../libyarn/src/libyarnclient/LibYarnClient.cpp |   12 +-
 .../libyarn/src/libyarnclient/LibYarnClient.h   |    6 +-
 .../src/libyarnclient/LibYarnClientC.cpp        |    4 +-
 .../libyarnserver/ApplicationClientProtocol.h   |    2 +-
 .../libyarnserver/ApplicationMasterProtocol.h   |    2 +-
 .../libyarnserver/ContainerManagementProtocol.h |    2 +-
 .../GetApplicationReportRequest.cpp             |    6 +-
 .../GetApplicationReportRequest.h               |    6 +-
 .../GetApplicationReportResponse.cpp            |    2 +-
 .../GetApplicationReportResponse.h              |    2 +-
 .../GetNewApplicationResponse.cpp               |    6 +-
 .../protocolrecords/GetNewApplicationResponse.h |    6 +-
 .../protocolrecords/KillApplicationRequest.cpp  |    2 +-
 .../protocolrecords/KillApplicationRequest.h    |    4 +-
 .../src/records/ApplicationAttemptId.cpp        |    6 +-
 .../libyarn/src/records/ApplicationAttemptId.h  |    6 +-
 depends/libyarn/src/records/ApplicationID.cpp   |   20 +-
 depends/libyarn/src/records/ApplicationID.h     |   56 -
 depends/libyarn/src/records/ApplicationId.h     |   56 +
 .../libyarn/src/records/ApplicationReport.cpp   |    8 +-
 depends/libyarn/src/records/ApplicationReport.h |    8 +-
 .../records/ApplicationSubmissionContext.cpp    |    6 +-
 .../src/records/ApplicationSubmissionContext.h  |    6 +-
 depends/libyarn/src/records/ContainerId.cpp     |    6 +-
 depends/libyarn/src/records/ContainerId.h       |    6 +-
 depends/libyarn/src/records/ContainerReport.cpp |    2 +-
 depends/libyarn/src/records/ContainerReport.h   |    2 +-
 depends/libyarn/src/records/ContainerStatus.cpp |    2 +-
 depends/libyarn/src/records/ContainerStatus.h   |    2 +-
 .../libyarn/test/function/TestLibYarnClient.cpp |    2 +
 .../test/function/TestLibYarnClientC.cpp        |  139 +++
 .../TestMockApplicationClientProtocol.cpp       |    2 +
 .../TestMockApplicationMasterProtocol.cpp       |    2 +
 .../libyarn/test/unit/TestApplicationClient.cpp |  264 -----
 .../libyarn/test/unit/TestApplicationMaster.cpp |  193 ----
 .../test/unit/TestContainerManagement.cpp       |  146 ---
 depends/libyarn/test/unit/TestLibYarnClient.cpp |  651 -----------
 .../TestLibYarnClient/TestApplicationClient.cpp |  277 +++++
 .../TestLibYarnClient/TestApplicationMaster.cpp |  196 ++++
 .../TestContainerManagement.cpp                 |  158 +++
 .../TestLibYarnClient/TestLibYarnClient.cpp     | 1085 ++++++++++++++++++
 .../TestLibYarnClient/TestLibYarnClientC.cpp    |  278 +++++
 .../libyarn/test/unit/TestLibYarnClientC.cpp    |  276 -----
 .../TestApplicationClientProtocol.cpp           |  213 ++++
 .../TestApplicationMasterProtocol.cpp           |   96 ++
 .../TestContainerManagementProtocol.cpp         |  101 ++
 .../TestProtocolRecords/TestAllocateRequest.cpp |  104 ++
 .../TestAllocateResponse.cpp                    |   67 ++
 .../TestFinishApplicationMasterRequest.cpp      |   70 ++
 .../TestFinishApplicationMasterResponse.cpp     |   39 +
 .../TestGetApplicationReportRequest.cpp         |   47 +
 .../TestGetApplicationReportResponse.cpp        |   39 +
 .../TestGetApplicationsRequest.cpp              |   70 ++
 .../TestGetApplicationsResponse.cpp             |   47 +
 .../TestGetClusterMetricsRequest.cpp            |   39 +
 .../TestGetClusterMetricsResponse.cpp           |   39 +
 .../TestGetClusterNodesRequest.cpp              |   55 +
 .../TestGetClusterNodesResponse.cpp             |   39 +
 .../TestGetContainerStatusesRequest.cpp         |   56 +
 .../TestGetContainerStatusesResponse.cpp        |   57 +
 .../TestGetContainersRequest.cpp                |   52 +
 .../TestGetContainersResponse.cpp               |   39 +
 .../TestGetNewApplicationRequest.cpp            |   39 +
 .../TestGetNewApplicationResponse.cpp           |   44 +
 .../TestGetQueueInfoRequest.cpp                 |   85 ++
 .../TestGetQueueInfoResponse.cpp                |   39 +
 .../TestGetQueueUserAclsInfoRequest.cpp         |   39 +
 .../TestGetQueueUserAclsInfoResponse.cpp        |   47 +
 .../TestKillApplicationRequest.cpp              |   39 +
 .../TestKillApplicationResponse.cpp             |   39 +
 .../TestRegisterApplicationMasterRequest.cpp    |   75 ++
 .../TestRegisterApplicationMasterResponse.cpp   |   39 +
 .../TestStartContainerRequest.cpp               |   65 ++
 .../TestStartContainerResponse.cpp              |   39 +
 .../TestStartContainersRequest.cpp              |   57 +
 .../TestStartContainersResponse.cpp             |   74 ++
 .../TestStopContainersRequest.cpp               |   57 +
 .../TestStopContainersResponse.cpp              |   74 ++
 .../TestSubmitApplicationRequest.cpp            |   52 +
 .../TestSubmitApplicationResponse.cpp           |   48 +
 .../TestRecords/TestApplicationAttemptId.cpp    |   56 +
 .../unit/TestRecords/TestApplicationReport.cpp  |  152 +++
 .../TestApplicationResourceUsageReport.cpp      |   98 ++
 .../TestApplicationSubmisionContext.cpp         |  156 +++
 .../TestRecords/TestContainerExceptionMap.cpp   |   59 +
 .../test/unit/TestRecords/TestContainerId.cpp   |   59 +
 .../TestRecords/TestContainerLaunchContext.cpp  |  124 ++
 .../unit/TestRecords/TestContainerReport.cpp    |  114 ++
 .../test/unit/TestRecords/TestLocalResource.cpp |  119 ++
 .../test/unit/TestRecords/TestNodeReport.cpp    |   81 ++
 .../TestRecords/TestPreemptionContainer.cpp     |   82 ++
 .../unit/TestRecords/TestPreemptionContract.cpp |   84 ++
 .../unit/TestRecords/TestPreemptionMessage.cpp  |   75 ++
 .../TestPreemptionResourceRequest.cpp           |   45 +
 .../TestResourceBlacklistRequest.cpp            |   88 ++
 .../TestRecords/TestSerializedException.cpp     |   81 ++
 .../TestStrictPreemptionContract.cpp            |   50 +
 .../unit/TestRecords/TestStringBytesMap.cpp     |   56 +
 .../TestRecords/TestStringLocalResourceMap.cpp  |   59 +
 .../unit/TestRecords/TestStringStringMap.cpp    |   56 +
 .../libyarn/test/unit/TestRecords/TestURL.cpp   |   89 ++
 110 files changed, 6436 insertions(+), 1854 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/mock/MockApplicationClient.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/mock/MockApplicationClient.h b/depends/libyarn/mock/MockApplicationClient.h
index 0c59a85..d5ba972 100644
--- a/depends/libyarn/mock/MockApplicationClient.h
+++ b/depends/libyarn/mock/MockApplicationClient.h
@@ -40,16 +40,17 @@ public:
     }
     ~MockApplicationClient(){
     }
-    MOCK_METHOD0(getNewApplication, ApplicationID ());
+    MOCK_METHOD0(getNewApplication, ApplicationId ());
     MOCK_METHOD1(submitApplication, void (ApplicationSubmissionContext &appContext));
-    MOCK_METHOD1(getApplicationReport, ApplicationReport (ApplicationID &appId));
+    MOCK_METHOD1(getApplicationReport, ApplicationReport (ApplicationId &appId));
     MOCK_METHOD1(getContainers, list<ContainerReport> (ApplicationAttemptId &appAttempId));
     MOCK_METHOD1(getClusterNodes, list<NodeReport> (list<NodeState> &state));
     MOCK_METHOD4(getQueueInfo, QueueInfo (string &queue, bool includeApps,bool includeChildQueues, bool recursive));
-    MOCK_METHOD1(forceKillApplication, void (ApplicationID &appId));
+    MOCK_METHOD1(forceKillApplication, void (ApplicationId &appId));
     MOCK_METHOD0(getClusterMetrics, YarnClusterMetrics ());
     MOCK_METHOD2(getApplications, list<ApplicationReport> (list<string> &applicationTypes,list<YarnApplicationState> &applicationStates));
     MOCK_METHOD0(getQueueAclsInfo, list<QueueUserACLInfo> ());
+    MOCK_METHOD0(getMethod, const AuthMethod ());
 };
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/mock/MockApplicationClientProtocolInternal.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/mock/MockApplicationClientProtocolInternal.h b/depends/libyarn/mock/MockApplicationClientProtocolInternal.h
new file mode 100644
index 0000000..4d84e1d
--- /dev/null
+++ b/depends/libyarn/mock/MockApplicationClientProtocolInternal.h
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+/*
+ * MockApplicationClientProtocolInternal.h
+ *
+ *  Created on: April 27, 2016
+ *      Author: Yangcheng Luo
+ */
+
+#ifndef MOCKAPPLICATIONCLIENTPROTOCOLINTERNAL_H_
+#define MOCKAPPLICATIONCLIENTPROTOCOLINTERNAL_H_
+#include <string>
+
+#include "gmock/gmock.h"
+#include "libyarnserver/ApplicationClientProtocol.h"
+
+using namespace libyarn;
+using std::string;
+
+namespace Mock{
+class MockApplicationClientProtocolInternal : public ApplicationClientProtocol {
+public:
+	MockApplicationClientProtocolInternal(const string & user, const string & nmHost, const string & nmPort,
+			const string & tokenService, const SessionConfig & c):
+			ApplicationClientProtocol(user, nmHost, nmPort, tokenService, c){
+	}
+	~MockApplicationClientProtocolInternal(){
+	}
+
+	MOCK_METHOD1(invoke, void(const RpcCall & call));
+
+};
+}
+#endif /* MOCKAPPLICATIONCLIENTPROTOCOLINTERNAL_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/mock/MockApplicationMasterProtocolInternal.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/mock/MockApplicationMasterProtocolInternal.h b/depends/libyarn/mock/MockApplicationMasterProtocolInternal.h
new file mode 100644
index 0000000..5fcaee3
--- /dev/null
+++ b/depends/libyarn/mock/MockApplicationMasterProtocolInternal.h
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+/*
+ * MockApplicationMasterProtocolInternal.h
+ *
+ *  Created on: May 4, 2016
+ *      Author: Yangcheng Luo
+ */
+
+#ifndef MOCKAPPLICATIONMASTERPROTOCOLINTERNAL_H_
+#define MOCKAPPLICATIONMASTERPROTOCOLINTERNAL_H_
+#include <string>
+
+#include "gmock/gmock.h"
+#include "libyarnserver/ApplicationMasterProtocol.h"
+
+using namespace libyarn;
+using std::string;
+
+namespace Mock{
+class MockApplicationMasterProtocolInternal : public ApplicationMasterProtocol {
+public:
+	MockApplicationMasterProtocolInternal(const string & schedHost, const string & schedPort,
+			const string & tokenService, const SessionConfig & c, const RpcAuth & a):
+			ApplicationMasterProtocol(schedHost, schedPort, tokenService, c, a){
+	}
+	~MockApplicationMasterProtocolInternal(){
+	}
+
+	MOCK_METHOD1(invoke, void(const RpcCall & call));
+
+};
+}
+#endif /* MOCKAPPLICATIONCLIENTPROTOCOLINTERNAL_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/mock/MockContainerManagementProtocolInternal.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/mock/MockContainerManagementProtocolInternal.h b/depends/libyarn/mock/MockContainerManagementProtocolInternal.h
new file mode 100644
index 0000000..aacf26c
--- /dev/null
+++ b/depends/libyarn/mock/MockContainerManagementProtocolInternal.h
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+/*
+ * MockContainerManagementProtocolInternal.h
+ *
+ *  Created on: April 22, 2016
+ *      Author: Ruilong Huo
+ */
+
+#ifndef MOCKCONTAINERMANAGEMENTPROTOCOLINTERNAL_H_
+#define MOCKCONTAINERMANAGEMENTPROTOCOLINTERNAL_H_
+#include <string>
+
+#include "gmock/gmock.h"
+#include "libyarnserver/ContainerManagementProtocol.h"
+
+using namespace libyarn;
+using std::string; using std::list;
+
+namespace Mock{
+class MockContainerManagementProtocolInternal : public ContainerManagementProtocol {
+public:
+	MockContainerManagementProtocolInternal(string & nmHost, string & nmPort,
+			const string & tokenService, const SessionConfig & c,
+			const RpcAuth & a):
+			ContainerManagementProtocol(nmHost,nmPort,tokenService, c,a){
+	}
+	~MockContainerManagementProtocolInternal(){
+	}
+
+	MOCK_METHOD1(invoke, void(const RpcCall & call));
+
+};
+}
+#endif /* MOCKCONTAINERMANAGEMENTPROTOCOLINTERNAL_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/CMakeLists.txt b/depends/libyarn/src/CMakeLists.txt
index cafe999..9fe2fcb 100644
--- a/depends/libyarn/src/CMakeLists.txt
+++ b/depends/libyarn/src/CMakeLists.txt
@@ -76,7 +76,7 @@ SET(HEADER
 
 SET(RECORDS_HEADER 
    	records/ApplicationAttemptId.h
-    records/ApplicationID.h
+    records/ApplicationId.h
     records/ApplicationReport.h
     records/ApplicationResourceUsageReport.h
     records/FinalApplicationStatus.h

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/common/Exception.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/common/Exception.cpp b/depends/libyarn/src/common/Exception.cpp
index 09d1312..dffe1cc 100644
--- a/depends/libyarn/src/common/Exception.cpp
+++ b/depends/libyarn/src/common/Exception.cpp
@@ -26,47 +26,21 @@ namespace Yarn {
 
 const char * YarnIOException::ReflexName = "java.io.IOException";
 
-const char * AlreadyBeingCreatedException::ReflexName =
-    "org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException";
-
 const char * AccessControlException::ReflexName =
     "org.apache.hadoop.security.AccessControlException";
 
-const char * FileAlreadyExistsException::ReflexName =
-    "org.apache.hadoop.fs.FileAlreadyExistsException";
-
-const char * DSQuotaExceededException::ReflexName =
-    "org.apache.hadoop.hdfs.protocol.DSQuotaExceededException";
-
-const char * NSQuotaExceededException::ReflexName =
-    "org.apache.hadoop.hdfs.protocol.NSQuotaExceededException";
-
-const char * ParentNotDirectoryException::ReflexName =
-    "org.apache.hadoop.fs.ParentNotDirectoryException";
-
 const char * SafeModeException::ReflexName =
     "org.apache.hadoop.hdfs.server.namenode.SafeModeException";
 
-const char * NotReplicatedYetException::ReflexName =
-    "org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException";
-
-const char * FileNotFoundException::ReflexName = "java.io.FileNotFoundException";
-
 const char * UnresolvedLinkException::ReflexName =
     "org.apache.hadoop.fs.UnresolvedLinkException";
 
 const char * UnsupportedOperationException::ReflexName =
     "java.lang.UnsupportedOperationException";
 
-const char * ReplicaNotFoundException::ReflexName =
-    "org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException";
-
 const char * ResourceManagerStandbyException::ReflexName =
     "org.apache.hadoop.ipc.StandbyException";
 
-const char * YarnInvalidBlockToken::ReflexName =
-    "org.apache.hadoop.security.token.SecretManager$InvalidToken";
-
 const char * SaslException::ReflexName = "javax.security.sasl.SaslException";
 
 const char * ApplicationMasterNotRegisteredException::ReflexName =

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/common/Exception.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/common/Exception.h b/depends/libyarn/src/common/Exception.h
index 46af910..ea54e0d 100644
--- a/depends/libyarn/src/common/Exception.h
+++ b/depends/libyarn/src/common/Exception.h
@@ -102,73 +102,6 @@ public:
     static const char * ReflexName;
 };
 
-class AlreadyBeingCreatedException: public YarnException {
-public:
-    AlreadyBeingCreatedException(const std::string & arg, const char * file,
-                                 int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~AlreadyBeingCreatedException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class ChecksumException: public YarnException {
-public:
-    ChecksumException(const std::string & arg, const char * file, int line,
-                      const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~ChecksumException() throw () {
-    }
-};
-
-class DSQuotaExceededException: public YarnException {
-public:
-    DSQuotaExceededException(const std::string & arg, const char * file,
-                             int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~DSQuotaExceededException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class FileAlreadyExistsException: public YarnException {
-public:
-    FileAlreadyExistsException(const std::string & arg, const char * file,
-                               int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~FileAlreadyExistsException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class FileNotFoundException: public YarnException {
-public:
-    FileNotFoundException(const std::string & arg, const char * file, int line,
-                          const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~FileNotFoundException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
 class YarnBadBoolFormat: public YarnException {
 public:
     YarnBadBoolFormat(const std::string & arg, const char * file, int line,
@@ -213,17 +146,6 @@ public:
     }
 };
 
-class YarnFileSystemClosed: public YarnException {
-public:
-    YarnFileSystemClosed(const std::string & arg, const char * file, int line,
-                         const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~YarnFileSystemClosed() throw () {
-    }
-};
-
 class YarnConfigInvalid: public YarnException {
 public:
     YarnConfigInvalid(const std::string & arg, const char * file, int line,
@@ -257,20 +179,6 @@ public:
     }
 };
 
-class YarnInvalidBlockToken: public YarnException {
-public:
-    YarnInvalidBlockToken(const std::string & arg, const char * file, int line,
-                          const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~YarnInvalidBlockToken() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
 /**
  * This will wrap YarnNetworkConnectionException and YarnTimeoutException.
  * This exception will be caught and attempt will be performed to recover in HA case.
@@ -357,73 +265,6 @@ public:
     }
 };
 
-class InvalidPath: public YarnException {
-public:
-    InvalidPath(const std::string & arg, const char * file, int line,
-                const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~InvalidPath() throw () {
-    }
-};
-
-class NotReplicatedYetException: public YarnException {
-public:
-    NotReplicatedYetException(const std::string & arg, const char * file,
-                              int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~NotReplicatedYetException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class NSQuotaExceededException: public YarnException {
-public:
-    NSQuotaExceededException(const std::string & arg, const char * file,
-                             int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~NSQuotaExceededException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class ParentNotDirectoryException: public YarnException {
-public:
-    ParentNotDirectoryException(const std::string & arg, const char * file,
-                                int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~ParentNotDirectoryException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
-class ReplicaNotFoundException: public YarnException {
-public:
-    ReplicaNotFoundException(const std::string & arg, const char * file,
-                             int line, const char * stack) :
-        YarnException(arg, file, line, stack) {
-    }
-
-    ~ReplicaNotFoundException() throw () {
-    }
-
-public:
-    static const char * ReflexName;
-};
-
 class SafeModeException: public YarnException {
 public:
     SafeModeException(const std::string & arg, const char * file, int line,

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnclient/ApplicationClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnclient/ApplicationClient.cpp b/depends/libyarn/src/libyarnclient/ApplicationClient.cpp
index ecbaf44..64acbe9 100644
--- a/depends/libyarn/src/libyarnclient/ApplicationClient.cpp
+++ b/depends/libyarn/src/libyarnclient/ApplicationClient.cpp
@@ -206,7 +206,7 @@ static void HandleYarnFailoverException(const Yarn::YarnFailoverException & e) {
  optional ResourceProto maximumCapability = 2;
  }
  */
-ApplicationID ApplicationClient::getNewApplication() {
+ApplicationId ApplicationClient::getNewApplication() {
     GetNewApplicationRequest request;
     GetNewApplicationResponse response;
 
@@ -250,7 +250,7 @@ void ApplicationClient::submitApplication(
  */
 
 ApplicationReport ApplicationClient::getApplicationReport(
-        ApplicationID &appId) {
+        ApplicationId &appId) {
     GetApplicationReportRequest request;
     GetApplicationReportResponse response;
 
@@ -302,7 +302,7 @@ QueueInfo ApplicationClient::getQueueInfo(string &queue, bool includeApps,
     return response.getQueueInfo();
 }
 
-void ApplicationClient::forceKillApplication(ApplicationID &appId) {
+void ApplicationClient::forceKillApplication(ApplicationId &appId) {
     KillApplicationRequest request;
     request.setApplicationId(appId);
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnclient/ApplicationClient.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnclient/ApplicationClient.h b/depends/libyarn/src/libyarnclient/ApplicationClient.h
index 4dd1b51..f078fa3 100644
--- a/depends/libyarn/src/libyarnclient/ApplicationClient.h
+++ b/depends/libyarn/src/libyarnclient/ApplicationClient.h
@@ -23,7 +23,7 @@
 #include <list>
 
 #include "libyarnserver/ApplicationClientProtocol.h"
-#include "records/ApplicationID.h"
+#include "records/ApplicationId.h"
 #include "records/ApplicationReport.h"
 #include "records/ContainerReport.h"
 #include "records/ApplicationSubmissionContext.h"
@@ -122,11 +122,11 @@ public:
 
     virtual ~ApplicationClient();
 
-    virtual ApplicationID getNewApplication();
+    virtual ApplicationId getNewApplication();
 
     virtual void submitApplication(ApplicationSubmissionContext &appContext);
 
-    virtual ApplicationReport getApplicationReport(ApplicationID &appId);
+    virtual ApplicationReport getApplicationReport(ApplicationId &appId);
 
     virtual list<ContainerReport> getContainers(ApplicationAttemptId &appAttempId);
 
@@ -135,7 +135,7 @@ public:
     virtual QueueInfo getQueueInfo(string &queue, bool includeApps,
             bool includeChildQueues, bool recursive);
 
-    virtual void forceKillApplication(ApplicationID &appId);
+    virtual void forceKillApplication(ApplicationId &appId);
 
     virtual YarnClusterMetrics getClusterMetrics();
 
@@ -146,7 +146,7 @@ public:
 
     const std::string & getUser(){uint32_t old=0; return getActiveAppClientProto(old)->getUser();};
 
-    const AuthMethod getMethod(){uint32_t old=0; return getActiveAppClientProto(old)->getMethod();};
+    virtual const AuthMethod getMethod(){uint32_t old=0; return getActiveAppClientProto(old)->getMethod();};
 
     const std::string getPrincipal(){uint32_t old=0; return getActiveAppClientProto(old)->getPrincipal();};
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnclient/LibYarnClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnclient/LibYarnClient.cpp b/depends/libyarn/src/libyarnclient/LibYarnClient.cpp
index 0cf90fa..823d116 100644
--- a/depends/libyarn/src/libyarnclient/LibYarnClient.cpp
+++ b/depends/libyarn/src/libyarnclient/LibYarnClient.cpp
@@ -160,7 +160,7 @@ int LibYarnClient::createJob(string &jobName, string &queue,string &jobId) {
         ApplicationClient *applicationClient = (ApplicationClient*)appClient;
 
         //1. getNewApplication
-        ApplicationID appId = applicationClient->getNewApplication();
+        ApplicationId appId = applicationClient->getNewApplication();
         LOG(INFO, "LibYarnClient::createJob, getNewApplication finished, appId:[clusterTimeStamp:%lld,id:%d]",
                 appId.getClusterTimestamp(), appId.getId());
 
@@ -425,13 +425,13 @@ void LibYarnClient::addResourceRequest(Resource capability,
 int LibYarnClient::addContainerRequests(string &jobId, Resource &capability, int32_t num_containers,
 									   list<LibYarnNodeInfo> &preferred, int32_t priority, bool relax_locality)
 {
-	if (jobId != clientJobId) {
-		throw std::invalid_argument("The jobId is wrong, check the jobId argument");
-	}
+	try {
+		if (jobId != clientJobId) {
+			throw std::invalid_argument("The jobId is wrong, check the jobId argument");
+		}
 
-	map<string, int32_t> inferredRacks;
+		map<string, int32_t> inferredRacks;
 
-	try {
 		for (list<LibYarnNodeInfo>::iterator iter = preferred.begin();
 				iter != preferred.end(); iter++) {
 			LOG(INFO, "LibYarnClient::addContainerRequests, "

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnclient/LibYarnClient.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnclient/LibYarnClient.h b/depends/libyarn/src/libyarnclient/LibYarnClient.h
index 2e3481c..f2c98ae 100644
--- a/depends/libyarn/src/libyarnclient/LibYarnClient.h
+++ b/depends/libyarn/src/libyarnclient/LibYarnClient.h
@@ -124,7 +124,7 @@ namespace libyarn {
 		//ContainerManagement
 		void *nmClient;
 
-		ApplicationID clientAppId;
+		ApplicationId clientAppId;
 		ApplicationAttemptId clientAppAttempId;
 
 		// the user of running AM, default is postgres
@@ -161,9 +161,9 @@ namespace libyarn {
 
 	class LibYarnNodeInfo {
 		public:
-			LibYarnNodeInfo(char *host, char* rack, int32_t cnt)
+			LibYarnNodeInfo(const string &host, const string &rack, int32_t cnt)
 							:hostname(host), num_containers(cnt)
-			{ if(rack == NULL) rackname = DEFAULT_RACK; else rackname = string(rack);}
+			{ if(rack == "") rackname = DEFAULT_RACK; else rackname = rack;}
 
 			string getHost() { return hostname; }
 			string getRack() { return rackname; }

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnclient/LibYarnClientC.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnclient/LibYarnClientC.cpp b/depends/libyarn/src/libyarnclient/LibYarnClientC.cpp
index cc7544d..a635cec 100644
--- a/depends/libyarn/src/libyarnclient/LibYarnClientC.cpp
+++ b/depends/libyarn/src/libyarnclient/LibYarnClientC.cpp
@@ -224,8 +224,8 @@ extern "C" {
 		list<LibYarnNodeInfo> preferredHostsList;
 		if (preferredHosts != NULL && preferredHostSize > 0) {
 			for (i = 0; i < preferredHostSize; i++) {
-				LibYarnNodeInfo *info = new LibYarnNodeInfo(preferredHosts[i].hostname,
-															preferredHosts[i].rackname,
+				LibYarnNodeInfo *info = new LibYarnNodeInfo(string(preferredHosts[i].hostname),
+															string(preferredHosts[i].rackname),
 															preferredHosts[i].num_containers);
 				preferredHostsList.push_back(*info);
 				totalPreferred += preferredHosts[i].num_containers;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnserver/ApplicationClientProtocol.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnserver/ApplicationClientProtocol.h b/depends/libyarn/src/libyarnserver/ApplicationClientProtocol.h
index 3a317c3..fb4497c 100644
--- a/depends/libyarn/src/libyarnserver/ApplicationClientProtocol.h
+++ b/depends/libyarn/src/libyarnserver/ApplicationClientProtocol.h
@@ -99,7 +99,7 @@ public:
 	const string getPrincipal() {return auth.getUser().getPrincipal();};
 
 private:
-	void invoke(const RpcCall & call);
+	virtual void invoke(const RpcCall & call);
 
 private:
 	RpcAuth auth;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnserver/ApplicationMasterProtocol.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnserver/ApplicationMasterProtocol.h b/depends/libyarn/src/libyarnserver/ApplicationMasterProtocol.h
index 157563e..e589a8a 100644
--- a/depends/libyarn/src/libyarnserver/ApplicationMasterProtocol.h
+++ b/depends/libyarn/src/libyarnserver/ApplicationMasterProtocol.h
@@ -73,7 +73,7 @@ public:
 			FinishApplicationMasterRequest &request);
 
 private:
-	void invoke(const RpcCall & call);
+	virtual void invoke(const RpcCall & call);
 
 private:
     RpcAuth auth;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/libyarnserver/ContainerManagementProtocol.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/libyarnserver/ContainerManagementProtocol.h b/depends/libyarn/src/libyarnserver/ContainerManagementProtocol.h
index 6ed3b51..e7ff9c7 100644
--- a/depends/libyarn/src/libyarnserver/ContainerManagementProtocol.h
+++ b/depends/libyarn/src/libyarnserver/ContainerManagementProtocol.h
@@ -71,7 +71,7 @@ public:
 	virtual GetContainerStatusesResponse getContainerStatuses(GetContainerStatusesRequest &request);
 
 private:
-	void invoke(const RpcCall & call);
+	virtual void invoke(const RpcCall & call);
 
 private:
 	RpcAuth auth;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.cpp b/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.cpp
index 5b1c710..558c8c8 100644
--- a/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.cpp
+++ b/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.cpp
@@ -37,14 +37,14 @@ GetApplicationReportRequestProto& GetApplicationReportRequest::getProto() {
 	return requestProto;
 }
 
-void GetApplicationReportRequest::setApplicationId(ApplicationID &appId) {
+void GetApplicationReportRequest::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto* proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	requestProto.set_allocated_application_id(proto);
 }
 
-ApplicationID GetApplicationReportRequest::getApplicationId() {
-	return ApplicationID(requestProto.application_id());
+ApplicationId GetApplicationReportRequest::getApplicationId() {
+	return ApplicationId(requestProto.application_id());
 }
 
 } /* namespace libyarn */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.h b/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.h
index 3caa47e..abbc6d5 100644
--- a/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.h
+++ b/depends/libyarn/src/protocolrecords/GetApplicationReportRequest.h
@@ -22,7 +22,7 @@
 
 #include "YARN_yarn_service_protos.pb.h"
 #include "YARN_yarn_protos.pb.h"
-#include "records/ApplicationID.h"
+#include "records/ApplicationId.h"
 
 using namespace hadoop::yarn;
 
@@ -42,8 +42,8 @@ public:
 
 	GetApplicationReportRequestProto& getProto();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 private:
 	GetApplicationReportRequestProto requestProto;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.cpp b/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.cpp
index 8c0f5c6..a07e0f5 100644
--- a/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.cpp
+++ b/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.cpp
@@ -33,7 +33,7 @@ GetApplicationReportResponse::GetApplicationReportResponse(
 GetApplicationReportResponse::~GetApplicationReportResponse() {
 }
 
-GetApplicationReportResponseProto& GetApplicationReportResponse::proto() {
+GetApplicationReportResponseProto& GetApplicationReportResponse::getProto() {
 	return responseProto;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.h b/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.h
index 237fe8a..bfbbaaa 100644
--- a/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.h
+++ b/depends/libyarn/src/protocolrecords/GetApplicationReportResponse.h
@@ -39,7 +39,7 @@ public:
 	GetApplicationReportResponse(const GetApplicationReportResponseProto &proto);
 	virtual ~GetApplicationReportResponse();
 
-	GetApplicationReportResponseProto& proto();
+	GetApplicationReportResponseProto& getProto();
 
 	void setApplicationReport(ApplicationReport &appReport);
 	ApplicationReport getApplicationReport();

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.cpp b/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.cpp
index 04e04ad..114f153 100644
--- a/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.cpp
+++ b/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.cpp
@@ -33,14 +33,14 @@ GetNewApplicationResponse::GetNewApplicationResponse(
 GetNewApplicationResponse::~GetNewApplicationResponse() {
 }
 
-void GetNewApplicationResponse::setApplicationId(ApplicationID &appId) {
+void GetNewApplicationResponse::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto *proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	responseProto.set_allocated_application_id(proto);
 }
 
-ApplicationID GetNewApplicationResponse::getApplicationId() {
-	return ApplicationID(responseProto.application_id());
+ApplicationId GetNewApplicationResponse::getApplicationId() {
+	return ApplicationId(responseProto.application_id());
 }
 
 void GetNewApplicationResponse::setResource(Resource &resource) {

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.h b/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.h
index 41335d6..12a7bd7 100644
--- a/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.h
+++ b/depends/libyarn/src/protocolrecords/GetNewApplicationResponse.h
@@ -22,7 +22,7 @@
 
 #include "YARN_yarn_protos.pb.h"
 #include "YARN_yarn_service_protos.pb.h"
-#include "records/ApplicationID.h"
+#include "records/ApplicationId.h"
 #include "records/Resource.h"
 
 using namespace hadoop::yarn;
@@ -40,8 +40,8 @@ public:
 	GetNewApplicationResponse(const GetNewApplicationResponseProto &proto);
 	virtual ~GetNewApplicationResponse();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 	void setResource(Resource &resource);
 	Resource getResource();

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/KillApplicationRequest.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/KillApplicationRequest.cpp b/depends/libyarn/src/protocolrecords/KillApplicationRequest.cpp
index 63ce33a..7702619 100644
--- a/depends/libyarn/src/protocolrecords/KillApplicationRequest.cpp
+++ b/depends/libyarn/src/protocolrecords/KillApplicationRequest.cpp
@@ -37,7 +37,7 @@ KillApplicationRequestProto& KillApplicationRequest::getProto() {
 	return requestProto;
 }
 
-void KillApplicationRequest::setApplicationId(ApplicationID &applicationId) {
+void KillApplicationRequest::setApplicationId(ApplicationId &applicationId) {
 	ApplicationIdProto* appId = new ApplicationIdProto();
 	appId->CopyFrom(applicationId.getProto());
 	requestProto.set_allocated_application_id(appId);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/protocolrecords/KillApplicationRequest.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/protocolrecords/KillApplicationRequest.h b/depends/libyarn/src/protocolrecords/KillApplicationRequest.h
index 6b9702b..3abd609 100644
--- a/depends/libyarn/src/protocolrecords/KillApplicationRequest.h
+++ b/depends/libyarn/src/protocolrecords/KillApplicationRequest.h
@@ -22,7 +22,7 @@
 
 #include "YARN_yarn_service_protos.pb.h"
 #include "YARN_yarn_protos.pb.h"
-#include "records/ApplicationID.h"
+#include "records/ApplicationId.h"
 
 using namespace hadoop::yarn;
 
@@ -39,7 +39,7 @@ public:
 	virtual ~KillApplicationRequest();
 
 	KillApplicationRequestProto& getProto();
-	void setApplicationId(ApplicationID &applicationId);
+	void setApplicationId(ApplicationId &applicationId);
 
 private:
 	KillApplicationRequestProto requestProto;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationAttemptId.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationAttemptId.cpp b/depends/libyarn/src/records/ApplicationAttemptId.cpp
index 1b47c07..8aab26e 100644
--- a/depends/libyarn/src/records/ApplicationAttemptId.cpp
+++ b/depends/libyarn/src/records/ApplicationAttemptId.cpp
@@ -36,14 +36,14 @@ ApplicationAttemptIdProto& ApplicationAttemptId::getProto(){
 	return attemptIdProto;
 }
 
-void ApplicationAttemptId::setApplicationId(ApplicationID &appId) {
+void ApplicationAttemptId::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto *proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	attemptIdProto.set_allocated_application_id(proto);
 }
 
-ApplicationID ApplicationAttemptId::getApplicationId() {
-	return ApplicationID(attemptIdProto.application_id());
+ApplicationId ApplicationAttemptId::getApplicationId() {
+	return ApplicationId(attemptIdProto.application_id());
 }
 
 void ApplicationAttemptId::setAttemptId(int32_t attemptId) {

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationAttemptId.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationAttemptId.h b/depends/libyarn/src/records/ApplicationAttemptId.h
index e9b70f9..9e4dafc 100644
--- a/depends/libyarn/src/records/ApplicationAttemptId.h
+++ b/depends/libyarn/src/records/ApplicationAttemptId.h
@@ -22,7 +22,7 @@
 #define APPLICATIONATTEMPTID_H_
 
 #include "YARN_yarn_protos.pb.h"
-#include "ApplicationID.h"
+#include "ApplicationId.h"
 
 using namespace hadoop::yarn;
 
@@ -41,8 +41,8 @@ public:
 
 	ApplicationAttemptIdProto& getProto();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 	void setAttemptId(int32_t attemptId);
 	int32_t getAttemptId();

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationID.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationID.cpp b/depends/libyarn/src/records/ApplicationID.cpp
index c17aa04..4fcff64 100644
--- a/depends/libyarn/src/records/ApplicationID.cpp
+++ b/depends/libyarn/src/records/ApplicationID.cpp
@@ -17,41 +17,41 @@
  * under the License.
  */
 
-#include "ApplicationID.h"
+#include "ApplicationId.h"
 
 namespace libyarn {
 
-ApplicationID::ApplicationID() {
+ApplicationId::ApplicationId() {
 	appIdProto = ApplicationIdProto::default_instance();
 }
 
-ApplicationID::ApplicationID(const ApplicationIdProto &proto) : appIdProto(proto) {
+ApplicationId::ApplicationId(const ApplicationIdProto &proto) : appIdProto(proto) {
 }
 
-ApplicationID::ApplicationID(const ApplicationID &applicationId){
+ApplicationId::ApplicationId(const ApplicationId &applicationId){
 	appIdProto = applicationId.appIdProto;
 }
 
-ApplicationID::~ApplicationID() {
+ApplicationId::~ApplicationId() {
 }
 
-ApplicationIdProto& ApplicationID::getProto() {
+ApplicationIdProto& ApplicationId::getProto() {
 	return appIdProto;
 }
 
-void ApplicationID::setId(int32_t id) {
+void ApplicationId::setId(int32_t id) {
 	appIdProto.set_id(id);
 }
 
-int ApplicationID::getId() {
+int ApplicationId::getId() {
 	return appIdProto.id();
 }
 
-void ApplicationID::setClusterTimestamp(int64_t timestamp) {
+void ApplicationId::setClusterTimestamp(int64_t timestamp) {
 	appIdProto.set_cluster_timestamp(timestamp);
 }
 
-int64_t ApplicationID::getClusterTimestamp() {
+int64_t ApplicationId::getClusterTimestamp() {
 	return appIdProto.cluster_timestamp();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationID.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationID.h b/depends/libyarn/src/records/ApplicationID.h
deleted file mode 100644
index 6eda5b3..0000000
--- a/depends/libyarn/src/records/ApplicationID.h
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef APPLICATIONIDS_H_
-#define APPLICATIONIDS_H_
-
-#include "YARN_yarn_protos.pb.h"
-
-using namespace hadoop::yarn;
-
-namespace libyarn {
-
-/*
-message ApplicationIdProto {
-  optional int32 id = 1;
-  optional int64 cluster_timestamp = 2;
-}
- */
-
-class ApplicationID {
-public:
-	ApplicationID();
-	ApplicationID(const ApplicationIdProto &proto);
-	ApplicationID(const ApplicationID &applicationId);
-	virtual ~ApplicationID();
-
-	ApplicationIdProto& getProto();
-
-	void setId(int32_t id);
-	int32_t getId();
-
-	void setClusterTimestamp(int64_t timestamp);
-	int64_t getClusterTimestamp();
-
-private:
-	ApplicationIdProto appIdProto;
-};
-
-} /* namespace libyarn */
-#endif /* APPLICATIONIDS_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationId.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationId.h b/depends/libyarn/src/records/ApplicationId.h
new file mode 100644
index 0000000..72014df
--- /dev/null
+++ b/depends/libyarn/src/records/ApplicationId.h
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef APPLICATIONIdS_H_
+#define APPLICATIONIdS_H_
+
+#include "YARN_yarn_protos.pb.h"
+
+using namespace hadoop::yarn;
+
+namespace libyarn {
+
+/*
+message ApplicationIdProto {
+  optional int32 id = 1;
+  optional int64 cluster_timestamp = 2;
+}
+ */
+
+class ApplicationId {
+public:
+	ApplicationId();
+	ApplicationId(const ApplicationIdProto &proto);
+	ApplicationId(const ApplicationId &applicationId);
+	virtual ~ApplicationId();
+
+	ApplicationIdProto& getProto();
+
+	void setId(int32_t id);
+	int32_t getId();
+
+	void setClusterTimestamp(int64_t timestamp);
+	int64_t getClusterTimestamp();
+
+private:
+	ApplicationIdProto appIdProto;
+};
+
+} /* namespace libyarn */
+#endif /* APPLICATIONIdS_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationReport.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationReport.cpp b/depends/libyarn/src/records/ApplicationReport.cpp
index 2ac14ef..93726e7 100644
--- a/depends/libyarn/src/records/ApplicationReport.cpp
+++ b/depends/libyarn/src/records/ApplicationReport.cpp
@@ -36,14 +36,14 @@ ApplicationReportProto& ApplicationReport::getProto() {
 	return reportProto;
 }
 
-void ApplicationReport::setApplicationId(ApplicationID &appId) {
+void ApplicationReport::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto *proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	reportProto.set_allocated_applicationid(proto);
 }
 
-ApplicationID ApplicationReport::getApplicationId() {
-	return ApplicationID(reportProto.applicationid());
+ApplicationId ApplicationReport::getApplicationId() {
+	return ApplicationId(reportProto.applicationid());
 }
 
 void ApplicationReport::setUser(string &user) {
@@ -136,7 +136,7 @@ int64_t ApplicationReport::getFinishTime() {
 	return reportProto.finishtime();
 }
 
-void ApplicationReport::setFinalAppStatus(FinalApplicationStatus status) {
+void ApplicationReport::setFinalApplicationStatus(FinalApplicationStatus status) {
 	reportProto.set_final_application_status((FinalApplicationStatusProto)status);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationReport.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationReport.h b/depends/libyarn/src/records/ApplicationReport.h
index 8aeb885..4106ac8 100644
--- a/depends/libyarn/src/records/ApplicationReport.h
+++ b/depends/libyarn/src/records/ApplicationReport.h
@@ -27,7 +27,7 @@
 
 #include "Token.h"
 #include "YarnApplicationState.h"
-#include "ApplicationID.h"
+#include "ApplicationId.h"
 #include "YarnApplicationState.h"
 #include "FinalApplicationStatus.h"
 #include "ApplicationResourceUsageReport.h"
@@ -69,8 +69,8 @@ public:
 
 	ApplicationReportProto& getProto();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 	void setUser(string &user);
 	string getUser();
@@ -105,7 +105,7 @@ public:
 	void setFinishTime(int64_t time);
 	int64_t getFinishTime();
 
-	void setFinalAppStatus(FinalApplicationStatus status);
+	void setFinalApplicationStatus(FinalApplicationStatus status);
 	FinalApplicationStatus getFinalApplicationStatus();
 
 	void setAppResourceUsage(ApplicationResourceUsageReport &usage);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationSubmissionContext.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationSubmissionContext.cpp b/depends/libyarn/src/records/ApplicationSubmissionContext.cpp
index da9e9b4..46284bd 100644
--- a/depends/libyarn/src/records/ApplicationSubmissionContext.cpp
+++ b/depends/libyarn/src/records/ApplicationSubmissionContext.cpp
@@ -37,14 +37,14 @@ ApplicationSubmissionContextProto& ApplicationSubmissionContext::getProto() {
 	return submitCtxProto;
 }
 
-void ApplicationSubmissionContext::setApplicationId(ApplicationID &appId) {
+void ApplicationSubmissionContext::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto *proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	submitCtxProto.set_allocated_application_id(proto);
 }
 
-ApplicationID ApplicationSubmissionContext::getApplicationId() {
-	return ApplicationID(submitCtxProto.application_id());
+ApplicationId ApplicationSubmissionContext::getApplicationId() {
+	return ApplicationId(submitCtxProto.application_id());
 }
 
 void ApplicationSubmissionContext::setApplicationName(string &applicationName) {

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ApplicationSubmissionContext.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ApplicationSubmissionContext.h b/depends/libyarn/src/records/ApplicationSubmissionContext.h
index f371a9f..28516a7 100644
--- a/depends/libyarn/src/records/ApplicationSubmissionContext.h
+++ b/depends/libyarn/src/records/ApplicationSubmissionContext.h
@@ -23,7 +23,7 @@
 #include <iostream>
 #include "ContainerLaunchContext.h"
 #include "YARN_yarn_protos.pb.h"
-#include "ApplicationID.h"
+#include "ApplicationId.h"
 #include "Priority.h"
 #include "Resource.h"
 
@@ -53,8 +53,8 @@ public:
 
 	ApplicationSubmissionContextProto& getProto();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 	void setApplicationName(string &applicationName);
 	string getApplicationName();

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerId.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerId.cpp b/depends/libyarn/src/records/ContainerId.cpp
index 6fabddc..d934b24 100644
--- a/depends/libyarn/src/records/ContainerId.cpp
+++ b/depends/libyarn/src/records/ContainerId.cpp
@@ -35,14 +35,14 @@ ContainerIdProto& ContainerId::getProto(){
 	return containerIdProto;
 }
 
-void ContainerId::setApplicationId(ApplicationID &appId) {
+void ContainerId::setApplicationId(ApplicationId &appId) {
 	ApplicationIdProto *proto = new ApplicationIdProto();
 	proto->CopyFrom(appId.getProto());
 	containerIdProto.set_allocated_app_id(proto);
 }
 
-ApplicationID ContainerId::getApplicationId() {
-	return ApplicationID(containerIdProto.app_id());
+ApplicationId ContainerId::getApplicationId() {
+	return ApplicationId(containerIdProto.app_id());
 }
 
 void ContainerId::setApplicationAttemptId(ApplicationAttemptId &appAttemptId) {

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerId.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerId.h b/depends/libyarn/src/records/ContainerId.h
index ee89d2a..9f5b8f7 100644
--- a/depends/libyarn/src/records/ContainerId.h
+++ b/depends/libyarn/src/records/ContainerId.h
@@ -22,7 +22,7 @@
 
 #include "YARN_yarn_protos.pb.h"
 
-#include "ApplicationID.h"
+#include "ApplicationId.h"
 #include "ApplicationAttemptId.h"
 
 using namespace hadoop::yarn;
@@ -45,8 +45,8 @@ public:
 
 	ContainerIdProto& getProto();
 
-	void setApplicationId(ApplicationID &appId);
-	ApplicationID getApplicationId();
+	void setApplicationId(ApplicationId &appId);
+	ApplicationId getApplicationId();
 
 	void setApplicationAttemptId(ApplicationAttemptId &appAttemptId);
 	ApplicationAttemptId getApplicationAttemptId();

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerReport.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerReport.cpp b/depends/libyarn/src/records/ContainerReport.cpp
index 9b4d7d2..1bded5e 100644
--- a/depends/libyarn/src/records/ContainerReport.cpp
+++ b/depends/libyarn/src/records/ContainerReport.cpp
@@ -97,7 +97,7 @@ ContainerExitStatus ContainerReport::getContainerExitStatus(){
 	return (ContainerExitStatus)reportProto.container_exit_status();
 }
 
-void ContainerReport::setContaierState(ContainerState state){
+void ContainerReport::setContainerState(ContainerState state){
 	reportProto.set_container_state((ContainerStateProto)state);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerReport.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerReport.h b/depends/libyarn/src/records/ContainerReport.h
index 2e034f0..0bf4ed5 100644
--- a/depends/libyarn/src/records/ContainerReport.h
+++ b/depends/libyarn/src/records/ContainerReport.h
@@ -79,7 +79,7 @@ public:
 	void setContainerExitStatus(ContainerExitStatus container_exit_status);
 	ContainerExitStatus getContainerExitStatus();
 
-	void setContaierState(ContainerState state);
+	void setContainerState(ContainerState state);
 	ContainerState getContainerState();
 
 	void setDiagnostics(string &diagnostics);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerStatus.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerStatus.cpp b/depends/libyarn/src/records/ContainerStatus.cpp
index ac9dab9..4048c55 100644
--- a/depends/libyarn/src/records/ContainerStatus.cpp
+++ b/depends/libyarn/src/records/ContainerStatus.cpp
@@ -45,7 +45,7 @@ ContainerId ContainerStatus::getContainerId() {
 	return ContainerId(statusProto.container_id());
 }
 
-void ContainerStatus::setContaierState(ContainerState state) {
+void ContainerStatus::setContainerState(ContainerState state) {
 	statusProto.set_state((ContainerStateProto)state);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/src/records/ContainerStatus.h
----------------------------------------------------------------------
diff --git a/depends/libyarn/src/records/ContainerStatus.h b/depends/libyarn/src/records/ContainerStatus.h
index f099985..59b3084 100644
--- a/depends/libyarn/src/records/ContainerStatus.h
+++ b/depends/libyarn/src/records/ContainerStatus.h
@@ -46,7 +46,7 @@ public:
 	void setContainerId(ContainerId &containerId);
 	ContainerId getContainerId();
 
-	void setContaierState(ContainerState state);
+	void setContainerState(ContainerState state);
 	ContainerState getContainerState();
 
 	void setDiagnostics(string &diagnostics);

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/function/TestLibYarnClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/function/TestLibYarnClient.cpp b/depends/libyarn/test/function/TestLibYarnClient.cpp
index 5260775..620c7d0 100644
--- a/depends/libyarn/test/function/TestLibYarnClient.cpp
+++ b/depends/libyarn/test/function/TestLibYarnClient.cpp
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+#include <string>
+
 #include "gtest/gtest.h"
 #include "libyarnclient/LibYarnClient.h"
 #include "records/FinalApplicationStatus.h"

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/function/TestLibYarnClientC.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/function/TestLibYarnClientC.cpp b/depends/libyarn/test/function/TestLibYarnClientC.cpp
new file mode 100644
index 0000000..2ec9743
--- /dev/null
+++ b/depends/libyarn/test/function/TestLibYarnClientC.cpp
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "gtest/gtest.h"
+#include "libyarn/LibYarnClientC.h"
+
+extern "C" {
+
+class TestLibYarnClientC: public ::testing::Test {
+public:
+	TestLibYarnClientC(){
+		char *user = "postgres";
+		char *rmHost = "localhost";
+		char *rmPort = "8032";
+		char *schedHost = "localhost";
+		char *schedPort = "8030";
+		char *amHost = "localhost";
+		int32_t amPort = 0;
+		char *am_tracking_url = "url";
+		int heartbeatInterval = 1000;
+		client = NULL;
+		result = newLibYarnClient(user, rmHost, rmPort, schedHost, schedPort, amHost, amPort, am_tracking_url, &client, heartbeatInterval);
+	}
+	~TestLibYarnClientC(){
+	}
+protected:
+	LibYarnClient_t *client;
+	int result;
+	int i;
+};
+
+TEST_F(TestLibYarnClientC,TestLibYarn){
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	char *jobName = "libyarn";
+	char *queue = "default";
+	char *jobId = NULL;
+	result = createJob(client, jobName, queue, &jobId);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	LibYarnNodeReport_t *nodeReportArray;
+	int nodeReportArraySize;
+	result = getClusterNodes(client, NODE_STATE_RUNNING, &nodeReportArray, &nodeReportArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+	EXPECT_GT(nodeReportArraySize, 0);
+
+	char *localhost = strdup(nodeReportArray[0].host);
+
+	freeMemNodeReportArray(nodeReportArray, nodeReportArraySize);
+
+	char *blackListAdditions[0];
+	char *blackListRemovals[0];
+	LibYarnNodeInfo_t preferredHosts[1];
+	preferredHosts[0].hostname = localhost;
+	preferredHosts[0].rackname = "/default-rack";
+	preferredHosts[0].num_containers = 2;
+	int preferredHostSize = 1;
+	LibYarnResource_t *allocatedResourcesArray;
+	int allocatedResourcesArraySize;
+	result = allocateResources(client, jobId, 1, 1, 1024, 5,
+					blackListAdditions, 0, blackListRemovals, 0, preferredHosts, preferredHostSize,
+					&allocatedResourcesArray, &allocatedResourcesArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	int64_t activeContainerIds[allocatedResourcesArraySize];
+	int64_t releaseContainerIds[allocatedResourcesArraySize];
+	int64_t statusContainerIds[allocatedResourcesArraySize];
+	for (i = 0 ; i < allocatedResourcesArraySize; i++) {
+		activeContainerIds[i] = allocatedResourcesArray[i].containerId;
+		releaseContainerIds[i] = allocatedResourcesArray[i].containerId;
+		statusContainerIds[i] = allocatedResourcesArray[i].containerId;
+	}
+
+	result = activeResources(client, jobId, activeContainerIds, allocatedResourcesArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	freeMemAllocatedResourcesArray(allocatedResourcesArray, allocatedResourcesArraySize);
+
+	sleep(10);
+
+	int64_t *activeFailIds;
+	int activeFailSize;
+	result = getActiveFailContainerIds(client,&activeFailIds,&activeFailSize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+	EXPECT_EQ(activeFailSize, 0);
+
+	LibYarnApplicationReport_t *applicationReport;
+	result = getApplicationReport(client, jobId, &applicationReport);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	freeApplicationReport(applicationReport);
+
+	LibYarnContainerReport_t *containerReportArray;
+	int containerReportArraySize;
+	result = getContainerReports(client, jobId, &containerReportArray, &containerReportArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+	EXPECT_EQ(containerReportArraySize, 5);
+
+	freeContainerReportArray(containerReportArray, containerReportArraySize);
+
+	LibYarnContainerStatus_t *containerStatusArray;
+	int containerStatusArraySize;
+	result = getContainerStatuses(client, jobId, statusContainerIds,
+					allocatedResourcesArraySize, &containerStatusArray,
+					&containerStatusArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+	EXPECT_EQ(containerReportArraySize, 5);
+
+	freeContainerStatusArray(containerStatusArray, containerStatusArraySize);
+
+	result = releaseResources(client, jobId, releaseContainerIds, allocatedResourcesArraySize);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	LibYarnQueueInfo_t *queueInfo = NULL;
+	result = getQueueInfo(client, queue, true, true, true, &queueInfo);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+
+	freeMemQueueInfo(queueInfo);
+
+	result = finishJob(client, jobId, APPLICATION_SUCCEEDED);
+	EXPECT_EQ(result, FUNCTION_SUCCEEDED);
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/function/TestMockApplicationClientProtocol.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/function/TestMockApplicationClientProtocol.cpp b/depends/libyarn/test/function/TestMockApplicationClientProtocol.cpp
index 24a110a..fa59c38 100644
--- a/depends/libyarn/test/function/TestMockApplicationClientProtocol.cpp
+++ b/depends/libyarn/test/function/TestMockApplicationClientProtocol.cpp
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+#include <string>
+
 #include "gtest/gtest.h"
 #include "gmock/gmock.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/function/TestMockApplicationMasterProtocol.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/function/TestMockApplicationMasterProtocol.cpp b/depends/libyarn/test/function/TestMockApplicationMasterProtocol.cpp
index 4236ccb..b48287b 100644
--- a/depends/libyarn/test/function/TestMockApplicationMasterProtocol.cpp
+++ b/depends/libyarn/test/function/TestMockApplicationMasterProtocol.cpp
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+#include <string>
+
 #include "gtest/gtest.h"
 #include "gmock/gmock.h"
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/unit/TestApplicationClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/unit/TestApplicationClient.cpp b/depends/libyarn/test/unit/TestApplicationClient.cpp
deleted file mode 100644
index d81ac0a..0000000
--- a/depends/libyarn/test/unit/TestApplicationClient.cpp
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#include "gtest/gtest.h"
-#include "gmock/gmock.h"
-
-#include "libyarnclient/ApplicationClient.h"
-#include "MockApplicationClientProtocol.h"
-
-using std::string;
-using std::list;
-using namespace libyarn;
-using namespace testing;
-using namespace Mock;
-
-class TestApplicationClient: public ::testing::Test {
-public:
-	TestApplicationClient(){
-		string user("postgres");
-		string rmHost("localhost");
-		string rmPort("8032");
-		string tokenService = "";
-		Yarn::Config config;
-		Yarn::Internal::SessionConfig sessionConfig(config);
-		MockApplicationClientProtocol *protocol = new MockApplicationClientProtocol(user,rmHost,rmPort,tokenService, sessionConfig);
-
-		ApplicationID appId;
-		appId.setId(100);
-		appId.setClusterTimestamp(1454307175682);
-		GetNewApplicationResponse getNewApplicationResponse;
-		getNewApplicationResponse.setApplicationId(appId);
-		EXPECT_CALL((*protocol),getNewApplication(_)).Times(AnyNumber()).WillOnce(Return(getNewApplicationResponse));
-		EXPECT_CALL((*protocol),submitApplication(_)).Times(AnyNumber()).WillOnce(Return());
-
-		ApplicationReport appReport;
-		appReport.setApplicationId(appId);
-		appReport.setUser(user);
-		string queue("default");
-		string appName("hawq");
-		string hostname("master");
-		appReport.setQueue(queue);
-		appReport.setName(appName);
-		appReport.setHost(hostname);
-		appReport.setRpcPort(8090);
-		appReport.setProgress(0.5);
-		GetApplicationReportResponse appReportResponse;
-		appReportResponse.setApplicationReport(appReport);
-		EXPECT_CALL((*protocol),getApplicationReport(_)).Times(AnyNumber()).WillOnce(Return(appReportResponse));
-
-		ContainerId containerId;
-		containerId.setId(501);
-		containerId.setApplicationId(appId);
-		Resource resource;
-		resource.setMemory(1024);
-		resource.setVirtualCores(1);
-		Priority priority;
-		priority.setPriority(1);
-		ContainerReport report;
-		report.setId(containerId);
-		report.setResource(resource);
-		report.setPriority(priority);
-		list<ContainerReport> reportList;
-		reportList.push_back(report);
-		GetContainersResponse getContainersResponse;
-		getContainersResponse.setContainersReportList(reportList);
-		EXPECT_CALL((*protocol),getContainers(_)).Times(AnyNumber()).WillOnce(Return(getContainersResponse));
-		
-		NodeId nodeId;
-		string nodeHost("node1");
-		nodeId.setHost(nodeHost);
-		nodeId.setPort(9983);
-		NodeReport nodeReport;
-		nodeReport.setNodeId(nodeId);
-		string rackName("default-rack");
-		nodeReport.setRackName(rackName);
-		nodeReport.setNumContainers(8);
-		Resource nodeResource;
-		nodeResource.setMemory(2048*8);
-		nodeResource.setVirtualCores(8);
-		nodeReport.setResourceCapablity(nodeResource);
-		nodeReport.setNodeState(NodeState::NS_RUNNING);
-		list<NodeReport> nodeReportList;
-		nodeReportList.push_back(nodeReport);
-		GetClusterNodesResponse getClusterNodesResponse;
-		getClusterNodesResponse.setNodeReports(nodeReportList);
-		EXPECT_CALL((*protocol),getClusterNodes(_)).Times(AnyNumber()).WillOnce(Return(getClusterNodesResponse));
-
-		QueueInfo queueInfo;
-		queueInfo.setQueueName(queue);
-		queueInfo.setCapacity(0.67);
-		queueInfo.setMaximumCapacity(0.95);
-		queueInfo.setCurrentCapacity(0.5);
-		queueInfo.setQueueState(QueueState::Q_RUNNING);
-		QueueInfo childQueue;
-		string childQueueName("hawq-queue");
-		childQueue.setQueueName(childQueueName);
-		childQueue.setCapacity(0.33);
-		childQueue.setMaximumCapacity(0.5);
-		childQueue.setCurrentCapacity(0.25);
-		list<QueueInfo> childQueueList;
-		childQueueList.push_back(childQueue);
-		queueInfo.setChildQueues(childQueueList);
-		list<ApplicationReport> appReportList;
-		appReportList.push_back(appReport);
-		queueInfo.setApplicationReports(appReportList);
-		GetQueueInfoResponse getQueueInfoResponse;
-		getQueueInfoResponse.setQueueInfo(queueInfo);
-		EXPECT_CALL((*protocol),getQueueInfo(_)).Times(AnyNumber()).WillOnce(Return(getQueueInfoResponse));
-
-		KillApplicationResponseProto killApplicationResponseProto;
-		EXPECT_CALL((*protocol),forceKillApplication(_)).Times(AnyNumber()).WillOnce(Return(KillApplicationResponse(killApplicationResponseProto)));
-
-		YarnClusterMetrics metrics;
-		metrics.setNumNodeManagers(10);
-		GetClusterMetricsResponse clusterMetricsResponse;
-		clusterMetricsResponse.setClusterMetrics(metrics);
-		EXPECT_CALL((*protocol),getClusterMetrics(_)).Times(AnyNumber()).WillOnce(Return(clusterMetricsResponse));
-	
-		GetApplicationsResponse applicationsResponse;
-		applicationsResponse.setApplicationList(appReportList);
-		EXPECT_CALL((*protocol),getApplications(_)).Times(AnyNumber()).WillOnce(Return(applicationsResponse));
-
-		QueueUserACLInfo aclInfo;
-		aclInfo.setQueueName(queue);
-		list<QueueACL> queueACLList;
-		QueueACL acl1 = QueueACL::QACL_ADMINISTER_QUEUE;
-		QueueACL acl2 = QueueACL::QACL_SUBMIT_APPLICATIONS;
-		queueACLList.push_back(acl1);
-		queueACLList.push_back(acl2);
-		aclInfo.setUserAcls(queueACLList);
-		list<QueueUserACLInfo> aclInfoList;
-		aclInfoList.push_back(aclInfo);
-		GetQueueUserAclsInfoResponse queueUserAclsInfoResponse;
-		queueUserAclsInfoResponse.setUserAclsInfoList(aclInfoList);
-		EXPECT_CALL((*protocol),getQueueAclsInfo(_)).Times(AnyNumber()).WillOnce(Return(queueUserAclsInfoResponse));
-
-		client = new ApplicationClient(protocol);
-	}
-	
-	~TestApplicationClient(){
-		delete client;
-	}
-	
-protected:
-	ApplicationClient *client;
-};
-
-TEST_F(TestApplicationClient, TestGetNewApplication){
-	ApplicationID response = client->getNewApplication();
-	EXPECT_EQ(response.getId(), 100);
-	EXPECT_EQ(response.getClusterTimestamp(), 1454307175682);
-}
-
-TEST_F(TestApplicationClient,TestSubmitApplication){
-	ApplicationSubmissionContext appContext;
-	client->submitApplication(appContext);
-}
-
-TEST_F(TestApplicationClient,TestGetApplicationReport){
-	ApplicationID appId;
-	ApplicationReport report = client->getApplicationReport(appId);
-	EXPECT_EQ(report.getUser(), "postgres");
-	EXPECT_EQ(report.getQueue(), "default");
-	EXPECT_EQ(report.getName(), "hawq");
-	EXPECT_EQ(report.getHost(), "master");
-	EXPECT_EQ(report.getRpcPort(), 8090);
-	EXPECT_FLOAT_EQ(report.getProgress(), 0.5); 
-}
-
-TEST_F(TestApplicationClient,TestGetContainers){
-	ApplicationAttemptId appAttempId;
-	list<ContainerReport> reports = client->getContainers(appAttempId);
-	EXPECT_EQ(reports.size(), 1);
-	list<ContainerReport>::iterator it = reports.begin();
-	EXPECT_EQ(it->getId().getId(), 501);
-	EXPECT_EQ(it->getPriority().getPriority(), 1);
-	EXPECT_EQ(it->getResource().getMemory(), 1024);
-	EXPECT_EQ(it->getResource().getVirtualCores(), 1);
-}
-
-TEST_F(TestApplicationClient,TestGetClusterNodes){
-	list<NodeState> states;
-	list<NodeReport> reports = client->getClusterNodes(states);
-	EXPECT_EQ(reports.size(), 1);
-	list<NodeReport>::iterator it = reports.begin();
-	EXPECT_EQ(it->getNodeId().getHost(), "node1");
-	EXPECT_EQ(it->getNodeId().getPort(), 9983);
-	EXPECT_EQ(it->getRackName(), "default-rack");
-	EXPECT_EQ(it->getResourceCapability().getMemory(), 2048*8);
-	EXPECT_EQ(it->getResourceCapability().getVirtualCores(), 8);
-	EXPECT_EQ(it->getNodeState(), NodeState::NS_RUNNING);
-	EXPECT_EQ(it->getNumContainers(), 8);
-}
-
-TEST_F(TestApplicationClient,TestGetQueueInfo){
-	string queue = "";
-	QueueInfo queueInfo = client->getQueueInfo(queue,true,true,true);
-	EXPECT_EQ(queueInfo.getQueueName(), "default");
-	EXPECT_FLOAT_EQ(queueInfo.getCapacity(), 0.67);
-	EXPECT_FLOAT_EQ(queueInfo.getMaximumCapacity(), 0.95);
-	EXPECT_FLOAT_EQ(queueInfo.getCurrentCapacity(), 0.5);
-	EXPECT_EQ(queueInfo.getQueueState(), QueueState::Q_RUNNING);
-	list<QueueInfo> child = queueInfo.getChildQueues();
-	EXPECT_EQ(child.size(), 1);
-	list<QueueInfo>::iterator it = child.begin();
-	EXPECT_EQ(it->getQueueName(), "hawq-queue");
-	EXPECT_FLOAT_EQ(it->getCapacity(), 0.33);
-	EXPECT_FLOAT_EQ(it->getMaximumCapacity(), 0.5);
-	EXPECT_FLOAT_EQ(it->getCurrentCapacity(), 0.25);
-	list<ApplicationReport> appReportList = queueInfo.getApplicationReports();
-	list<ApplicationReport>::iterator itAppReport = appReportList.begin();
-	EXPECT_EQ(itAppReport->getApplicationId().getId(), 100);
-	EXPECT_EQ(itAppReport->getUser(), "postgres");
-}
-
-TEST_F(TestApplicationClient,TestForceKillApplication){
-	ApplicationID appId;
-	client->forceKillApplication(appId);
-}
-
-TEST_F(TestApplicationClient,TestGetClusterMetrics){
-	YarnClusterMetrics response = client->getClusterMetrics();
-	EXPECT_EQ(response.getNumNodeManagers(), 10);
-}
-
-TEST_F(TestApplicationClient,TestGetApplications){
-	list<string> applicationTypes;
-	list<YarnApplicationState> applicationStates;
-	list<ApplicationReport> reports = client->getApplications(applicationTypes,applicationStates);
-	EXPECT_EQ(reports.size(), 1);
-	list<ApplicationReport>::iterator it = reports.begin();
-	EXPECT_EQ(it->getApplicationId().getId(), 100);
-	EXPECT_EQ(it->getUser(), "postgres");
-}
-
-TEST_F(TestApplicationClient,TestGetQueueAclsInfo){
-	list<QueueUserACLInfo> response = client->getQueueAclsInfo();
-	EXPECT_EQ(response.size(), 1);
-	list<QueueUserACLInfo>::iterator it = response.begin();
-	EXPECT_EQ(it->getQueueName(), "default");
-	list<QueueACL> queueACLs = it->getUserAcls();
-	EXPECT_EQ(queueACLs.size(), 2);
-	list<QueueACL>::iterator queueACL = queueACLs.begin();
-	EXPECT_EQ(*queueACL, QueueACL::QACL_ADMINISTER_QUEUE);
-	*queueACL++;
-	EXPECT_EQ(*queueACL, QueueACL::QACL_SUBMIT_APPLICATIONS);
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/383818c2/depends/libyarn/test/unit/TestApplicationMaster.cpp
----------------------------------------------------------------------
diff --git a/depends/libyarn/test/unit/TestApplicationMaster.cpp b/depends/libyarn/test/unit/TestApplicationMaster.cpp
deleted file mode 100644
index ad2f7d4..0000000
--- a/depends/libyarn/test/unit/TestApplicationMaster.cpp
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *   http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#include "gtest/gtest.h"
-#include "gmock/gmock.h"
-
-#include "libyarnclient/ApplicationMaster.h"
-#include "MockApplicationMasterProtocol.h"
-
-using std::string;
-using std::list;
-using namespace libyarn;
-using namespace testing;
-using namespace Mock;
-
-class TestApplicationMaster: public ::testing::Test {
-public:
-	TestApplicationMaster(){
-		string schedHost("localhost");
-		string schedPort("8032");
-		string tokenService = "";
-		Yarn::Config config;
-		Yarn::Internal::SessionConfig sessionConfig(config);
-		Yarn::Internal::UserInfo user = Yarn::Internal::UserInfo::LocalUser();
-		Yarn::Internal::RpcAuth rpcAuth(user, Yarn::Internal::AuthMethod::SIMPLE);
-		protocol = new MockApplicationMasterProtocol(schedHost,schedPort,tokenService, sessionConfig,rpcAuth);
-		client = new ApplicationMaster(protocol);
-	}
-	~TestApplicationMaster(){
-		delete client;
-	}
-
-protected:
-	MockApplicationMasterProtocol *protocol;
-	ApplicationMaster *client;
-};
-
-TEST_F(TestApplicationMaster,TestRegisterApplicationMaster){
-	Resource resource;
-	resource.setMemory(1024*8*10);
-	resource.setVirtualCores(1*8*10);
-	string key("tokenkey");
-	ApplicationACLMap aclMap;
-	aclMap.setAccessType(ApplicationAccessType::APPACCESS_VIEW_APP);
-	string acl("acl");
-	aclMap.setAcl(acl);
-	list<ApplicationACLMap> aclMapList;
-	aclMapList.push_back(aclMap);
-	RegisterApplicationMasterResponse response;
-	response.setMaximumResourceCapability(resource);
-	response.setClientToAMTokenMasterKey(key);
-	response.setApplicationACLs(aclMapList);
-	EXPECT_CALL((*protocol),registerApplicationMaster(_)).Times(AnyNumber()).WillOnce(Return(response));
-	
-	string amHost("localhost");
-	int amPort = 8032;
-	string am_tracking_url = "";
-	RegisterApplicationMasterResponse retResponse = client->registerApplicationMaster(amHost,amPort,am_tracking_url);
-	EXPECT_EQ(retResponse.getClientToAMTokenMasterKey(), "tokenkey");
-	Resource retResource = retResponse.getMaximumResourceCapability();
-	EXPECT_EQ(retResource.getMemory(), 1024*8*10);
-	EXPECT_EQ(retResource.getVirtualCores(), 1*8*10);
-	list<ApplicationACLMap> retAclMapList = retResponse.getApplicationACLs();
-	EXPECT_EQ(retAclMapList.size(), 1);
-	list<ApplicationACLMap>::iterator it = retAclMapList.begin();
-	EXPECT_EQ(it->getAccessType(), ApplicationAccessType::APPACCESS_VIEW_APP);
-	EXPECT_EQ(it->getAcl(), "acl");
-}
-
-TEST_F(TestApplicationMaster,TestAllocate){
-	Resource resource;
-	resource.setMemory(1024*8*10);
-	resource.setVirtualCores(1*8*10);
-	AllocateResponse allocateResponse;
-	allocateResponse.setAMCommand(AMCommand::AM_RESYNC);
-	allocateResponse.setResponseId(100);
-	list<Container> containers;
-	Container container;
-	ContainerId containerId;
-	containerId.setId(501);
-	container.setId(containerId);
-	NodeId nodeId;
-	string nodeHost("node1");
-	nodeId.setHost(nodeHost);
-	nodeId.setPort(9983);
-	container.setNodeId(nodeId);
-	string address("http://address");
-	container.setNodeHttpAddress(address);
-	container.setResource(resource);
-	Priority priority;
-	priority.setPriority(1);
-	container.setPriority(priority);
-	libyarn::Token token;
-	string identifier("identifier");
-	token.setIdentifier(identifier);
-	string password("password");
-	token.setPassword(password);
-	string kind("kind");
-	token.setKind(kind);
-	string service("service");
-	token.setService(service);
-	container.setContainerToken(token);
-	containers.push_back(container);
-	allocateResponse.setAllocatedContainers(containers);
-	ContainerStatus containerStatus;
-	containerStatus.setContainerId(containerId);
-	containerStatus.setContaierState(ContainerState::C_RUNNING);
-	string diagnostics("diagnostics");
-	containerStatus.setDiagnostics(diagnostics);
-	containerStatus.setExitStatus(-1000);
-	list<ContainerStatus> statuses;
-	statuses.push_back(containerStatus);
-	allocateResponse.setCompletedContainerStatuses(statuses);
-	allocateResponse.setResourceLimit(resource);
-	NodeReport nodeReport;
-	nodeReport.setNodeId(nodeId);
-	string rackName("default-rack");
-	nodeReport.setRackName(rackName);
-	nodeReport.setNumContainers(8);
-	list<NodeReport> nodeReports;
-	nodeReports.push_back(nodeReport);
-	allocateResponse.setUpdatedNodes(nodeReports);
-	allocateResponse.setNumClusterNodes(12);
-	NMToken nmToken;
-	nmToken.setNodeId(nodeId);
-	nmToken.setToken(token);
-	list<NMToken> nmTokens;
-	nmTokens.push_back(nmToken);
-	allocateResponse.setNMTokens(nmTokens);
-	EXPECT_CALL((*protocol),allocate(_)).Times(AnyNumber()).WillOnce(Return(allocateResponse));
-		
-	list<ResourceRequest> asks;
-	list<ContainerId> releases;
-	ResourceBlacklistRequest blacklistRequest;
-	int32_t responseId;
-	float progress = 5;
-	AllocateResponse retResponse = client->allocate(asks,releases,blacklistRequest,responseId,progress);
-	EXPECT_EQ(retResponse.getAMCommand(), AMCommand::AM_RESYNC);
-	EXPECT_EQ(retResponse.getResponseId(), 100);
-	list<Container> retContainers = retResponse.getAllocatedContainers();
-	list<Container>::iterator it = retContainers.begin();
-	EXPECT_EQ(it->getId().getId(), 501);
-	EXPECT_EQ(it->getNodeId().getHost(), "node1");
-	EXPECT_EQ(it->getNodeId().getPort(), 9983);
-	EXPECT_EQ(it->getNodeHttpAddress(), "http://address");
-	EXPECT_EQ(it->getPriority().getPriority(), 1);
-	EXPECT_EQ(it->getResource().getMemory(), 1024*8*10);
-	EXPECT_EQ(it->getResource().getVirtualCores(), 1*8*10);
-	EXPECT_EQ(it->getContainerToken().getIdentifier(), "identifier");
-	EXPECT_EQ(it->getContainerToken().getPassword(), "password");
-	EXPECT_EQ(it->getContainerToken().getKind(), "kind");
-	EXPECT_EQ(it->getContainerToken().getService(), "service");
-	list<ContainerStatus>::iterator retStatus = retResponse.getCompletedContainersStatuses().begin();
-	EXPECT_EQ(retStatus->getContainerId().getId(), 501);
-	EXPECT_EQ(retStatus->getContainerState(), ContainerState::C_RUNNING);
-	//EXPECT_EQ(retStatus->getDiagnostics(), "diagnostics");
-	EXPECT_EQ(retStatus->getExitStatus(), -1000);
-	EXPECT_EQ(retResponse.getResourceLimit().getMemory(), 1024*8*10);
-	//list<NodeReport>::iterator report = response.getUpdatedNodes().begin();
-	//EXPECT_EQ(report->getNodeId().getHost(), "node1");
-	//list<NMToken>::iterator nmToken = response.getNMTokens().begin();
-	//EXPECT_EQ(nmToken->getNodeId().getHost(), "node1");
-	//EXPECT_EQ(nmToken->getToken().getIdentifier(), "identifier");
-	EXPECT_EQ(retResponse.getNumClusterNodes(), 12);
-}
-
-TEST_F(TestApplicationMaster,TestFinishApplicationMaster){
-	FinishApplicationMasterResponse finishApplicationMasterResponse;
-	finishApplicationMasterResponse.setIsUnregistered(true);
-	EXPECT_CALL((*protocol),finishApplicationMaster(_)).Times(AnyNumber()).WillOnce(Return(finishApplicationMasterResponse));
-	string diagnostics("");
-	string trackingUrl("");
-	FinalApplicationStatus finalstatus;
-	bool response = client->finishApplicationMaster(diagnostics,trackingUrl,finalstatus);
-	EXPECT_EQ(response,true);
-}
-