You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by rl...@apache.org on 2017/07/24 08:53:22 UTC

[4/4] incubator-hawq git commit: HAWQ-1500. Support TDE by adding a common class HttpClient to get response from KMS.

HAWQ-1500. Support TDE by adding a common class HttpClient to get response from KMS.


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

Branch: refs/heads/master
Commit: 04b8884eee2a2f95cf0a24accb19dd77f441f045
Parents: 993a918
Author: amyrazz44 <ab...@pivotal.io>
Authored: Tue Jul 11 15:43:18 2017 +0800
Committer: rlei <rl...@pivotal.io>
Committed: Mon Jul 24 16:53:15 2017 +0800

----------------------------------------------------------------------
 depends/libhdfs3/CMake/FindCurl.cmake         |  26 ++
 depends/libhdfs3/CMake/FindSSL.cmake          |  26 ++
 depends/libhdfs3/CMakeLists.txt               |   2 +
 depends/libhdfs3/mock/MockHttpClient.h        |  53 ++++
 depends/libhdfs3/src/CMakeLists.txt           |   6 +
 depends/libhdfs3/src/client/HttpClient.cpp    | 337 +++++++++++++++++++++
 depends/libhdfs3/src/client/HttpClient.h      | 155 ++++++++++
 depends/libhdfs3/src/client/Permission.cpp    |   5 +-
 depends/libhdfs3/src/client/UserInfo.h        |   4 +
 depends/libhdfs3/src/common/SessionConfig.cpp |  15 +-
 depends/libhdfs3/src/common/SessionConfig.h   |  27 +-
 depends/libhdfs3/test/function/CMakeLists.txt |   4 +
 depends/libhdfs3/test/unit/CMakeLists.txt     |   4 +
 13 files changed, 659 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/CMake/FindCurl.cmake
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/CMake/FindCurl.cmake b/depends/libhdfs3/CMake/FindCurl.cmake
new file mode 100644
index 0000000..e93b01d
--- /dev/null
+++ b/depends/libhdfs3/CMake/FindCurl.cmake
@@ -0,0 +1,26 @@
+# - Try to find the CURL library (curl)
+#
+# Once done this will define
+#
+#  CURL_FOUND - System has gnutls
+#  CURL_INCLUDE_DIR - The gnutls include directory
+#  CURL_LIBRARIES - The libraries needed to use gnutls
+#  CURL_DEFINITIONS - Compiler switches required for using gnutls
+
+
+IF (CURL_INCLUDE_DIR AND CURL_LIBRARIES)
+	# in cache already
+	SET(CURL_FIND_QUIETLY TRUE)
+ENDIF (CURL_INCLUDE_DIR AND CURL_LIBRARIES)
+
+FIND_PATH(CURL_INCLUDE_DIR curl/curl.h)
+
+FIND_LIBRARY(CURL_LIBRARIES curl)
+
+INCLUDE(FindPackageHandleStandardArgs)
+
+# handle the QUIETLY and REQUIRED arguments and set CURL_FOUND to TRUE if 
+# all listed variables are TRUE
+FIND_PACKAGE_HANDLE_STANDARD_ARGS(CURL DEFAULT_MSG CURL_LIBRARIES CURL_INCLUDE_DIR)
+
+MARK_AS_ADVANCED(CURL_INCLUDE_DIR CURL_LIBRARIES)

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/CMake/FindSSL.cmake
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/CMake/FindSSL.cmake b/depends/libhdfs3/CMake/FindSSL.cmake
new file mode 100644
index 0000000..bcbc5d8
--- /dev/null
+++ b/depends/libhdfs3/CMake/FindSSL.cmake
@@ -0,0 +1,26 @@
+# - Try to find the Open ssl library (ssl)
+#
+# Once done this will define
+#
+#  SSL_FOUND - System has gnutls
+#  SSL_INCLUDE_DIR - The gnutls include directory
+#  SSL_LIBRARIES - The libraries needed to use gnutls
+#  SSL_DEFINITIONS - Compiler switches required for using gnutls
+
+
+IF (SSL_INCLUDE_DIR AND SSL_LIBRARIES)
+	# in cache already
+	SET(SSL_FIND_QUIETLY TRUE)
+ENDIF (SSL_INCLUDE_DIR AND SSL_LIBRARIES)
+
+FIND_PATH(SSL_INCLUDE_DIR openssl/opensslv.h)
+
+FIND_LIBRARY(SSL_LIBRARIES crypto)
+
+INCLUDE(FindPackageHandleStandardArgs)
+
+# handle the QUIETLY and REQUIRED arguments and set SSL_FOUND to TRUE if
+# all listed variables are TRUE
+FIND_PACKAGE_HANDLE_STANDARD_ARGS(SSL DEFAULT_MSG SSL_LIBRARIES SSL_INCLUDE_DIR)
+
+MARK_AS_ADVANCED(SSL_INCLUDE_DIR SSL_LIBRARIES)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/CMakeLists.txt b/depends/libhdfs3/CMakeLists.txt
index d787ed8..4694942 100644
--- a/depends/libhdfs3/CMakeLists.txt
+++ b/depends/libhdfs3/CMakeLists.txt
@@ -21,6 +21,8 @@ FIND_PACKAGE(Protobuf REQUIRED)
 FIND_PACKAGE(KERBEROS REQUIRED)
 FIND_PACKAGE(GSasl REQUIRED)
 FIND_PACKAGE(GoogleTest REQUIRED)
+FIND_PACKAGE(SSL REQUIRED)
+FIND_PACKAGE(CURL REQUIRED)
 INCLUDE_DIRECTORIES(${GoogleTest_INCLUDE_DIR})
 LINK_LIBRARIES(${GoogleTest_LIBRARIES})
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/mock/MockHttpClient.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/mock/MockHttpClient.h b/depends/libhdfs3/mock/MockHttpClient.h
new file mode 100644
index 0000000..d0e1fd4
--- /dev/null
+++ b/depends/libhdfs3/mock/MockHttpClient.h
@@ -0,0 +1,53 @@
+/********************************************************************
+ * 2014 - 
+ * open source under Apache License Version 2.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 _HDFS_LIBHDFS3_MOCK_HTTPCLIENT_H_
+#define _HDFS_LIBHDFS3_MOCK_HTTPCLIENT_H_
+
+#include "gmock/gmock.h"
+
+#include "client/HttpClient.h"
+#include "client/KmsClientProvider.h"
+#include <boost/property_tree/ptree.hpp>
+
+using boost::property_tree::ptree;
+
+class MockHttpClient: public Hdfs::HttpClient {
+public:
+  MOCK_METHOD0(post, std::string());
+  MOCK_METHOD0(del, std::string());
+  MOCK_METHOD0(put, std::string());
+  MOCK_METHOD0(get, std::string());
+
+  std::string getPostResult(FileEncryptionInfo &encryptionInfo) {
+	ptree map;
+	map.put("name", encryptionInfo.getKeyName());
+	map.put("iv", encryptionInfo.getIv());
+	map.put("material", encryptionInfo.getKey());
+
+	std::string json = KmsClientProvider::toJson(map);
+	return json;
+  }
+
+
+};
+
+#endif /* _HDFS_LIBHDFS3_MOCK_HTTPCLIENT_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/CMakeLists.txt b/depends/libhdfs3/src/CMakeLists.txt
index bc20d08..f9e2817 100644
--- a/depends/libhdfs3/src/CMakeLists.txt
+++ b/depends/libhdfs3/src/CMakeLists.txt
@@ -91,16 +91,22 @@ INCLUDE_DIRECTORIES(${LIBXML2_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${KERBEROS_INCLUDE_DIRS})
 INCLUDE_DIRECTORIES(${GSASL_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/mock)
+INCLUDE_DIRECTORIES(${SSL_INCLUDE_DIR})
+INCLUDE_DIRECTORIES(${CURL_INCLUDE_DIR})
 
 TARGET_LINK_LIBRARIES(libhdfs3-static ${PROTOBUF_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${LIBXML2_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${KERBEROS_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-static ${GSASL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-static ${SSL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-static ${CURL_LIBRARIES})
 
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${PROTOBUF_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${LIBXML2_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${KERBEROS_LIBRARIES})
 TARGET_LINK_LIBRARIES(libhdfs3-shared ${GSASL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-shared ${SSL_LIBRARIES})
+TARGET_LINK_LIBRARIES(libhdfs3-shared ${CURL_LIBRARIES})
 
 SET_TARGET_PROPERTIES(libhdfs3-static PROPERTIES OUTPUT_NAME "hdfs3")
 SET_TARGET_PROPERTIES(libhdfs3-shared PROPERTIES OUTPUT_NAME "hdfs3")

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/client/HttpClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/HttpClient.cpp b/depends/libhdfs3/src/client/HttpClient.cpp
new file mode 100644
index 0000000..6a80a99
--- /dev/null
+++ b/depends/libhdfs3/src/client/HttpClient.cpp
@@ -0,0 +1,337 @@
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.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 "HttpClient.h"
+#include "Logger.h"
+
+using namespace Hdfs::Internal;
+
+namespace Hdfs {
+
+#define CURL_SETOPT(handle, option, optarg, fmt, ...) \
+    res = curl_easy_setopt(handle, option, optarg); \
+    if (res != CURLE_OK) { \
+        THROW(HdfsIOException, fmt, ##__VA_ARGS__); \
+    }
+
+#define CURL_SETOPT_ERROR1(handle, option, optarg, fmt) \
+    CURL_SETOPT(handle, option, optarg, fmt, curl_easy_strerror(res));
+
+#define CURL_SETOPT_ERROR2(handle, option, optarg, fmt) \
+    CURL_SETOPT(handle, option, optarg, fmt, curl_easy_strerror(res), \
+        errorString().c_str())
+
+#define CURL_PERFORM(handle, fmt) \
+    res = curl_easy_perform(handle); \
+    if (res != CURLE_OK) { \
+        THROW(HdfsIOException, fmt, curl_easy_strerror(res), errorString().c_str()); \
+    }
+
+
+#define CURL_GETOPT_ERROR2(handle, option, optarg, fmt) \
+    res = curl_easy_getinfo(handle, option, optarg); \
+    if (res != CURLE_OK) { \
+        THROW(HdfsIOException, fmt, curl_easy_strerror(res), errorString().c_str()); \
+    }
+
+#define CURL_GET_RESPONSE(handle, code, fmt) \
+    CURL_GETOPT_ERROR2(handle, CURLINFO_RESPONSE_CODE, code, fmt);
+
+HttpClient::HttpClient() : curl(NULL), list(NULL) {
+
+}
+
+/**
+ * Construct a HttpClient instance.
+ * @param url a url which is the address to send the request to the corresponding http server.
+ */
+HttpClient::HttpClient(const std::string &url) {
+	curl = NULL;
+	list = NULL;
+	this->url = url;
+}
+
+/**
+ * Destroy a HttpClient instance.
+ */
+HttpClient::~HttpClient()
+{
+	destroy();
+}
+
+/**
+ * Receive error string from curl.
+ */
+std::string HttpClient::errorString() {
+	if (strlen(errbuf) == 0)
+		return "";
+	return errbuf;
+}
+
+/**
+ * Curl call back function to receive the reponse messages.
+ * @return return the size of reponse messages. 
+ */
+size_t HttpClient::CurlWriteMemoryCallback(void *contents, size_t size, size_t nmemb, void *userp)
+{
+      size_t realsize = size * nmemb;
+	  if (userp == NULL || contents == NULL) {
+		  return 0;
+	  }
+      ((std::string *)userp)->append((const char *)contents, realsize);
+	  LOG(DEBUG2, "HttpClient : Http response is : %s", ((std::string *)userp)->c_str());
+      return realsize;
+}
+
+/**
+ * Init curl handler and set curl options.
+ */
+void HttpClient::init() {
+	if (!initialized)
+	{
+		initialized = true;
+		if (curl_global_init(CURL_GLOBAL_ALL)) {
+			THROW(HdfsIOException, "Cannot initialize curl client for KMS");
+		}
+	}
+
+	curl = curl_easy_init();
+	if (!curl) {
+		THROW(HdfsIOException, "Cannot initialize curl handle for KMS");
+	}
+	
+    CURL_SETOPT_ERROR1(curl, CURLOPT_ERRORBUFFER, errbuf,
+        "Cannot initialize curl error buffer for KMS: %s");
+
+    errbuf[0] = 0;
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_NOPROGRESS, 1,
+        "Cannot initialize no progress in HttpClient: %s: %s");
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_VERBOSE, 0,
+        "Cannot initialize no verbose in HttpClient: %s: %s");
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_COOKIEFILE, "",
+        "Cannot initialize cookie behavior in HttpClient: %s: %s");
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_HTTPHEADER, list,
+        "Cannot initialize headers in HttpClient: %s: %s");
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_WRITEFUNCTION, HttpClient::CurlWriteMemoryCallback,
+        "Cannot initialize body reader in HttpClient: %s: %s");
+
+    CURL_SETOPT_ERROR2(curl, CURLOPT_WRITEDATA, (void *)&response,
+        "Cannot initialize body reader data in HttpClient: %s: %s");
+
+    /* Some servers don't like requests that are made without a user-agent
+        field, so we provide one */
+    CURL_SETOPT_ERROR2(curl, CURLOPT_USERAGENT, "libcurl-agent/1.0",
+        "Cannot initialize user agent in HttpClient: %s: %s");
+	list = NULL;
+
+}
+
+/**
+ * Do clean up for curl.
+ */
+void HttpClient::destroy() {
+	if (curl) {
+		curl_easy_cleanup(curl);
+	}
+	if (list) {
+		curl_slist_free_all(list);
+	}
+}
+
+/**
+ * Set url for http client.
+ */
+void HttpClient::setURL(const std::string &url) {
+	this->url = url;
+}
+
+/**
+ * Set retry times for http request which can be configured in config file.
+ */
+void HttpClient::setRequestRetryTimes(int request_retry_times) {
+	if (request_retry_times < 0) {
+		THROW(InvalidParameter, "HttpClient : Invalid value for request_retry_times.");
+	}
+	this->request_retry_times = request_retry_times;
+}
+
+/**
+ * Set request timeout which can be configured in config file.
+ */
+void HttpClient::setRequestTimeout(int64_t curl_timeout) {
+	if (curl_timeout < 0) {
+		THROW(InvalidParameter, "HttpClient : Invalid value for curl_timeout.");
+	}
+	this->curl_timeout = curl_timeout;
+}
+
+/**
+ * Set headers for http client.
+ */
+void HttpClient::setHeaders(const std::vector<std::string> &headers) {
+	if (!headers.empty()) {
+		this->headers = headers;
+		for (std::string header : headers) {
+			list = curl_slist_append(list, header.c_str());
+			if (!list) {
+				THROW(HdfsIOException, "Cannot add header in HttpClient.");
+			}
+		}	
+	}
+	else {
+		LOG(DEBUG1, "HttpClient : Header is empty.");				
+	}
+}
+
+
+/**
+ * Set body for http client.
+ */
+void HttpClient::setBody(const std::string &body) {
+	this->body = body;
+}
+
+/**
+ * Set expected response code.
+ */
+void HttpClient::setExpectedResponseCode(int64_t response_code_ok) {
+	this->response_code_ok = response_code_ok;
+}
+
+/**
+ * Http common method to get response info by sending request to http server.
+ * @param method : define different http methods.
+ * @return return response info.
+ */
+std::string HttpClient::httpCommon(httpMethod method) {
+	
+    /* Set headers and url. */
+	if (list != NULL) {
+		CURL_SETOPT_ERROR2(curl, CURLOPT_HTTPHEADER, list,
+        	        "Cannot initialize headers in HttpClient: %s: %s");
+	} else {
+		LOG(DEBUG1, "HttpClient : Http Header is NULL");
+	}
+
+	if (curl != NULL) {
+		CURL_SETOPT_ERROR2(curl, CURLOPT_URL, url.c_str(),
+    	        "Cannot initialize url in HttpClient: %s: %s");
+	} else {
+		LOG(DEBUG1, "HttpClient : Http URL is NULL");
+	}
+
+	/* Set body based on different http method. */ 
+	switch(method) {
+		case HTTP_GET:
+			break;
+		case HTTP_POST:
+			CURL_SETOPT_ERROR2(curl, CURLOPT_COPYPOSTFIELDS, body.c_str(),
+                "Cannot initialize post data in HttpClient: %s: %s");
+			break;
+		case HTTP_DELETE:
+			CURL_SETOPT_ERROR2(curl, CURLOPT_CUSTOMREQUEST, "DELETE",
+                "Cannot initialize set customer request in HttpClient: %s: %s");
+			break;
+		case HTTP_PUT:
+			CURL_SETOPT_ERROR2(curl, CURLOPT_CUSTOMREQUEST, "PUT",
+                "Cannot initialize set customer request in HttpClient: %s: %s");
+			CURL_SETOPT_ERROR2(curl, CURLOPT_COPYPOSTFIELDS, body.c_str(),
+                "Cannot initialize post data in HttpClient: %s: %s");
+			break;
+	}
+
+	/* Do several http request try according to request_retry_times until got the right reponse code. */	
+	int64_t response_code = -1;	
+
+	while (request_retry_times >= 0 && response_code != response_code_ok) {
+		request_retry_times -= 1;
+		response = "";
+		CURL_SETOPT_ERROR2(curl, CURLOPT_TIMEOUT, curl_timeout, 
+					"Send request to http server timeout: %s: %s");
+		CURL_PERFORM(curl, "Could not send request in HttpClient: %s %s");
+		CURL_GET_RESPONSE(curl, &response_code,
+					"Cannot get response code in HttpClient: %s: %s");
+	}
+	LOG(DEBUG1, "HttpClient : The http method is %d. The http url is %s. The http response is %s.", method, url.c_str(), response.c_str());
+	return response;
+}
+
+/**
+ * Http GET method.
+ */
+std::string HttpClient::get() {
+	httpMethod method = HTTP_GET;
+	return httpCommon(method);
+}
+
+/**
+ * Http POST method.
+ */
+std::string HttpClient::post() {
+	httpMethod method = HTTP_POST;
+	return httpCommon(method);
+}
+
+/**
+ * Http DELETE method.
+ */
+std::string HttpClient::del() {
+	httpMethod method = HTTP_DELETE;
+	return httpCommon(method);
+}
+
+/**
+ * Http PUT method.
+ */
+std::string HttpClient::put() {
+	httpMethod method = HTTP_PUT;
+	return httpCommon(method);
+}
+
+
+/**
+ *  URL encodes the given string. 
+ */	
+std::string HttpClient::escape(const std::string &data) {
+	if (curl) {
+		char *output = curl_easy_escape(curl, data.c_str(), data.length());
+		if (output) {
+			std::string out(output);
+			return out;
+		} else {
+			THROW(HdfsIOException, "HttpClient : Curl escape failed.");
+		}
+	} else {
+		LOG(WARNING, "HttpClient : Curl in escape method is NULL");
+	}
+
+}
+}
+
+/* Curl global init only can be done once. */
+bool Hdfs::HttpClient::initialized = false;
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/client/HttpClient.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/HttpClient.h b/depends/libhdfs3/src/client/HttpClient.h
new file mode 100644
index 0000000..9bada33
--- /dev/null
+++ b/depends/libhdfs3/src/client/HttpClient.h
@@ -0,0 +1,155 @@
+/********************************************************************
+ * 2014 -
+ * open source under Apache License Version 2.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 _HDFS_LIBHDFS3_CLIENT_HTTPCLIENT_H_
+#define _HDFS_LIBHDFS3_CLIENT_HTTPCLIENT_H_
+
+#include <string>
+#include <vector>
+#include <curl/curl.h>
+#include "Exception.h"
+#include "ExceptionInternal.h"
+
+typedef enum httpMethod {
+    HTTP_GET = 0,
+	HTTP_POST = 1,
+	HTTP_DELETE = 2,
+	HTTP_PUT = 3
+} httpMethod;
+
+namespace Hdfs {
+
+class HttpClient {
+public:
+	HttpClient();
+
+	/**
+	 * Construct a HttpClient instance.
+	 * @param url a url which is the address to send the request to the corresponding http server. 
+	 */
+	HttpClient(const std::string &url);
+
+	/**
+	 * Destroy a HttpClient instance.
+	 */
+	virtual ~HttpClient();
+	
+	/**
+	 * Set url for http client.
+	 */
+	void setURL(const std::string &url);	
+
+	/**
+	 * Set headers for http client.
+	 */
+	void setHeaders(const std::vector<std::string> &headers);
+	
+	/**
+	 * Set body for http client.
+	 */
+	void setBody(const std::string &body);	
+	
+	/**
+	 * Set retry times for http request which can be configured in config file.
+	 */
+	void setRequestRetryTimes(int requst_retry_times);	
+	
+	/**
+	 * Set request timeout which can be configured in config file.
+	 */
+	void setRequestTimeout(int64_t curl_timeout);	
+	
+	/**
+	 * Set expected response code.
+	 */
+	void setExpectedResponseCode(int64_t response_code_ok);
+
+	/**
+	 * Init curl handler and set options for curl.
+	 */
+	void init();
+	
+	/**
+	 * Do clean up for curl.
+	 */
+	void destroy();	
+	
+	/**
+	 * Http POST method. 
+	 */
+	virtual std::string post();
+	
+	/**
+	 * Http DELETE method. 
+	 */
+	virtual std::string del();
+	
+	/**
+	 * Http PUT method. 
+	 */
+	virtual std::string put();
+	
+	/**
+	 * Http GET method. 
+	 */
+	virtual std::string get();
+
+	/**
+	 * URL encodes the given string. 
+	 */
+	std::string escape(const std::string &data);
+
+	/**
+	 * Receive error string from curl. 
+	 */
+	std::string errorString();
+
+private:
+
+	/**
+ 	 * Http common method to get response info by sending request to http server.
+	 * @param method : define different http methods.
+	 * @return return response info.
+	 */
+ 	std::string 	httpCommon(httpMethod method);
+
+	/**
+	 * Curl call back function to receive the reponse messages.
+	 * @return return the size of reponse messages.
+	 */
+	static size_t 	CurlWriteMemoryCallback(void *contents, size_t size, size_t nmemb, void *userp); 
+
+	static bool initialized;
+	CURLcode res;
+	std::string url;
+	std::vector<std::string> headers;
+	std::string body;
+	int64_t response_code_ok;
+	int request_retry_times;
+	int64_t curl_timeout;	
+	CURL *curl;
+	struct curl_slist *list;
+	std::string response;
+	char errbuf[CURL_ERROR_SIZE] = {0};
+};
+
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/client/Permission.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/Permission.cpp b/depends/libhdfs3/src/client/Permission.cpp
index da5ec51..fedf226 100644
--- a/depends/libhdfs3/src/client/Permission.cpp
+++ b/depends/libhdfs3/src/client/Permission.cpp
@@ -27,7 +27,10 @@
 namespace Hdfs {
 
 Permission::Permission(uint16_t mode) {
-    if (mode >> 10) {
+	uint16_t fileEncryptionBit = (1 << 13);
+	bool isFileEncryption = (((mode & fileEncryptionBit) != 0) ? true : false);
+
+    if (!isFileEncryption && mode >> 10) {
         THROW(InvalidParameter,
               "Invalid parameter: cannot convert %u to \"Permission\"",
               static_cast<unsigned int>(mode));

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/client/UserInfo.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/UserInfo.h b/depends/libhdfs3/src/client/UserInfo.h
index 7262987..0489171 100644
--- a/depends/libhdfs3/src/client/UserInfo.h
+++ b/depends/libhdfs3/src/client/UserInfo.h
@@ -59,6 +59,10 @@ public:
         this->effectiveUser = KerberosName(effectiveUser);
     }
 
+	std::string getKrbName() const {
+	    return effectiveUser.getName();
+      
+	}
     std::string getPrincipal() const {
         return effectiveUser.getPrincipal();
     }

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/common/SessionConfig.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/SessionConfig.cpp b/depends/libhdfs3/src/common/SessionConfig.cpp
index 632009e..d531036 100644
--- a/depends/libhdfs3/src/common/SessionConfig.cpp
+++ b/depends/libhdfs3/src/common/SessionConfig.cpp
@@ -126,19 +126,28 @@ SessionConfig::SessionConfig(const Config & conf) {
             &socketCacheExpiry, "dfs.client.socketcache.expiryMsec", 3000, bind(CheckRangeGE<int32_t>, _1, _2, 0)
         }, {
             &socketCacheCapacity, "dfs.client.socketcache.capacity", 16, bind(CheckRangeGE<int32_t>, _1, _2, 0)
-        }
+        }, {
+			&cryptoBufferSize, "hadoop.security.crypto.buffer.size", 8192
+		},{
+			&httpRequestRetryTimes, "kms.send.request.retry.times", 4
+		}
     };
     ConfigDefault<int64_t> i64Values [] = {
         {
             &defaultBlockSize, "dfs.default.blocksize", 64 * 1024 * 1024, bind(CheckMultipleOf<int64_t>, _1, _2, 512)
-        }
+        }, {
+            &curlTimeout, "kms.send.request.timeout", 20L
+		}
     };
+
     ConfigDefault<std::string> strValues [] = {
         {&defaultUri, "dfs.default.uri", "hdfs://localhost:8020" },
         {&rpcAuthMethod, "hadoop.security.authentication", "simple" },
         {&kerberosCachePath, "hadoop.security.kerberos.ticket.cache.path", "" },
         {&logSeverity, "dfs.client.log.severity", "INFO" },
-        {&domainSocketPath, "dfs.domain.socket.path", ""}
+        {&domainSocketPath, "dfs.domain.socket.path", ""},
+		{&kmsUrl, "dfs.encryption.key.provider.uri", ""},
+		{&kmsAuthMethod, "hadoop.kms.authentication.type", "simple" }
     };
 
     for (size_t i = 0; i < ARRAYSIZE(boolValues); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/src/common/SessionConfig.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/common/SessionConfig.h b/depends/libhdfs3/src/common/SessionConfig.h
index 3ff9f19..020e99b 100644
--- a/depends/libhdfs3/src/common/SessionConfig.h
+++ b/depends/libhdfs3/src/common/SessionConfig.h
@@ -301,6 +301,26 @@ public:
       return socketCacheCapacity;
     }
 
+	const std::string& getKmsUrl() const {
+      return kmsUrl;
+    }
+
+	const std::string& getKmsMethod() const {
+        return kmsAuthMethod;
+    }
+
+	int32_t getCryptoBufferSize() const {
+        return cryptoBufferSize;
+    }
+
+	int32_t getHttpRequestRetryTimes() const {
+        return httpRequestRetryTimes;
+    }
+
+	int64_t getCurlTimeOut() const {
+        return curlTimeout;
+    }
+
 public:
     /*
      * rpc configure
@@ -359,7 +379,12 @@ public:
     int32_t packetPoolSize;
     int32_t heartBeatInterval;
     int32_t closeFileTimeout;
-
+	std::string kmsUrl;
+	std::string kmsAuthMethod;
+	int32_t cryptoBufferSize;
+	int32_t httpRequestRetryTimes;
+	int64_t curlTimeout;
+	
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/test/function/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/test/function/CMakeLists.txt b/depends/libhdfs3/test/function/CMakeLists.txt
index 8faa259..d1d28c8 100644
--- a/depends/libhdfs3/test/function/CMakeLists.txt
+++ b/depends/libhdfs3/test/function/CMakeLists.txt
@@ -16,6 +16,8 @@ INCLUDE_DIRECTORIES(${libhdfs3_PLATFORM_HEADER_DIR})
 INCLUDE_DIRECTORIES(${LIBXML2_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${KERBEROS_INCLUDE_DIRS})
 INCLUDE_DIRECTORIES(${GSASL_INCLUDE_DIR})
+INCLUDE_DIRECTORIES(${SSL_INCLUDE_DIR})
+INCLUDE_DIRECTORIES(${CURL_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/mock)
 
 PROTOBUF_GENERATE_CPP(libhdfs3_PROTO_SOURCES libhdfs3_PROTO_HEADERS ${libhdfs3_PROTO_FILES})
@@ -66,6 +68,8 @@ TARGET_LINK_LIBRARIES(function ${LIBXML2_LIBRARIES})
 TARGET_LINK_LIBRARIES(function ${KERBEROS_LIBRARIES})
 TARGET_LINK_LIBRARIES(function ${GSASL_LIBRARIES})
 TARGET_LINK_LIBRARIES(function ${GoogleTest_LIBRARIES})
+TARGET_LINK_LIBRARIES(function ${SSL_LIBRARIES})
+TARGET_LINK_LIBRARIES(function ${CURL_LIBRARIES})
 
 SET(function_SOURCES ${function_SOURCES} PARENT_SCOPE)
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/04b8884e/depends/libhdfs3/test/unit/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/test/unit/CMakeLists.txt b/depends/libhdfs3/test/unit/CMakeLists.txt
index 46f62ab..939f5fa 100644
--- a/depends/libhdfs3/test/unit/CMakeLists.txt
+++ b/depends/libhdfs3/test/unit/CMakeLists.txt
@@ -18,6 +18,8 @@ INCLUDE_DIRECTORIES(${LIBXML2_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${KERBEROS_INCLUDE_DIRS})
 INCLUDE_DIRECTORIES(${GSASL_INCLUDE_DIR})
 INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/mock)
+INCLUDE_DIRECTORIES(${SSL_INCLUDE_DIR})
+INCLUDE_DIRECTORIES(${CURL_INCLUDE_DIR})
 
 ADD_DEFINITIONS(-DMOCK)
 
@@ -34,6 +36,8 @@ ADD_EXECUTABLE(unit EXCLUDE_FROM_ALL
     ${unit_SOURCES}
 )
 
+TARGET_LINK_LIBRARIES(unit ${SSL_LIBRARIES})
+TARGET_LINK_LIBRARIES(unit ${CURL_LIBRARIES})
 TARGET_LINK_LIBRARIES(unit pthread)
 
 IF(NEED_BOOST)