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:21 UTC

[3/4] incubator-hawq git commit: HAWQ-1501. Support TDE by adding KmsClientProvider class to interact with KMS server.

HAWQ-1501. Support TDE by adding KmsClientProvider class to interact with KMS server.


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

Branch: refs/heads/master
Commit: c024842ff3102aae0611d468da4f36aedb7e3c07
Parents: 04b8884
Author: amyrazz44 <ab...@pivotal.io>
Authored: Tue Jul 11 15:51:32 2017 +0800
Committer: rlei <rl...@pivotal.io>
Committed: Mon Jul 24 16:53:15 2017 +0800

----------------------------------------------------------------------
 depends/libhdfs3/mock/MockKmsClientProvider.h   |  50 +++
 .../libhdfs3/src/client/KmsClientProvider.cpp   | 318 +++++++++++++++++++
 depends/libhdfs3/src/client/KmsClientProvider.h | 142 +++++++++
 .../libhdfs3/test/function/TestKmsClient.cpp    | 206 ++++++++++++
 4 files changed, 716 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c024842f/depends/libhdfs3/mock/MockKmsClientProvider.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/mock/MockKmsClientProvider.h b/depends/libhdfs3/mock/MockKmsClientProvider.h
new file mode 100644
index 0000000..e530230
--- /dev/null
+++ b/depends/libhdfs3/mock/MockKmsClientProvider.h
@@ -0,0 +1,50 @@
+/********************************************************************
+ * 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_KMSCLIENTPROVIDER_H_
+#define _HDFS_LIBHDFS3_MOCK_KMSCLIENTPROVIDER_H_
+
+#include "gmock/gmock.h"
+
+#include "client/KmsClientProvider.h"
+
+using namespace Hdfs::Internal;
+
+class MockKmsClientProvider: public Hdfs::KmsClientProvider {
+public:
+  MockKmsClientProvider(std::shared_ptr<RpcAuth> auth, std::shared_ptr<SessionConfig> conf) : KmsClientProvider(auth, conf) {}
+  MOCK_METHOD1(setHttpClient, void(std::shared_ptr<HttpClient> hc));
+  MOCK_METHOD1(getKeyMetadata, ptree(const FileEncryptionInfo &encryptionInfo));
+  MOCK_METHOD1(deleteKey, void(const FileEncryptionInfo &encryptionInfo));
+  MOCK_METHOD1(decryptEncryptedKey, ptree(const FileEncryptionInfo &encryptionInfo));
+  MOCK_METHOD5(createKey, void(const std::string &keyName, const std::string &cipher, const int length, const std::string &material, const std::string &description));
+
+ ptree getEDKResult(FileEncryptionInfo &encryptionInfo) {
+    ptree map;
+    map.put("name", encryptionInfo.getKeyName());
+    map.put("iv", encryptionInfo.getIv());
+    map.put("material", KmsClientProvider::base64Encode(encryptionInfo.getKey()));
+    return map;
+  }
+
+};
+
+#endif /* _HDFS_LIBHDFS3_MOCK_KMSCLIENTPROVIDER_H_ */

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c024842f/depends/libhdfs3/src/client/KmsClientProvider.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/KmsClientProvider.cpp b/depends/libhdfs3/src/client/KmsClientProvider.cpp
new file mode 100644
index 0000000..596b07f
--- /dev/null
+++ b/depends/libhdfs3/src/client/KmsClientProvider.cpp
@@ -0,0 +1,318 @@
+/********************************************************************
+ * 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 "KmsClientProvider.h"
+#include "Logger.h"
+#include <gsasl.h>
+#include <map>
+#include <boost/property_tree/json_parser.hpp>
+using namespace Hdfs::Internal;
+using boost::property_tree::read_json;
+using boost::property_tree::write_json;
+
+namespace Hdfs {
+
+/**
+ * Convert ptree format to json format
+ */
+std::string KmsClientProvider::toJson(const ptree &data)
+{
+	std::ostringstream buf;
+	try {
+		write_json(buf, data, false);
+		std::string json = buf.str();
+		return json;
+	} catch (...) {
+		THROW(HdfsIOException, "KmsClientProvider : Write json failed.");
+	}	
+}
+
+/**
+ * Convert json format to ptree format
+ */
+ptree KmsClientProvider::fromJson(const std::string &data)
+{
+	ptree pt2;
+	try {
+		std::istringstream is(data);
+		read_json(is, pt2);
+		return pt2;
+	} catch (...) {
+		THROW(HdfsIOException, "KmsClientProvider : Read json failed.");
+	}
+}
+
+/**
+ * Encode string to base64. 
+ */
+std::string	KmsClientProvider::base64Encode(const std::string &data)
+{
+	char * buffer = NULL;
+	size_t len = 0;
+	int rc = 0;
+	std::string result;
+
+	LOG(DEBUG1, "KmsClientProvider : Encode data is %s", data.c_str());
+
+	if (GSASL_OK != (rc = gsasl_base64_to(data.c_str(), data.size(), &buffer, &len))) {
+		assert(GSASL_MALLOC_ERROR == rc);
+        throw std::bad_alloc();
+	}
+
+	if (buffer) {
+		result.assign(buffer, len);
+		free(buffer);
+	} 
+
+	if (!buffer || result.length() != len) {
+        THROW(HdfsIOException, "KmsClientProvider: Failed to encode string to base64");
+    }
+
+	return result;	
+}
+
+/**
+ * Decode base64 to string.
+ */
+std::string	KmsClientProvider::base64Decode(const std::string &data)
+{
+	char * buffer = NULL;
+	size_t len = 0;
+	int rc = 0;
+	std::string result;
+
+	if (GSASL_OK != (rc = gsasl_base64_from(data.c_str(), data.size(), &buffer, &len))) {
+		assert(GSASL_MALLOC_ERROR == rc);
+        throw std::bad_alloc();
+	}
+
+	if (buffer) {
+		result.assign(buffer, len);
+		free(buffer);
+	} 
+
+	if (!buffer || result.length() != len) {
+        THROW(HdfsIOException, "KmsClientProvider: Failed to decode base64 to string");
+    }
+
+	return result;	
+}
+
+/**
+ * Construct a KmsClientProvider instance.
+ * @param auth RpcAuth to get the auth method and user info.
+ * @param conf a SessionConfig to get the configuration.
+ */
+KmsClientProvider::KmsClientProvider(std::shared_ptr<RpcAuth> rpcAuth, std::shared_ptr<SessionConfig> config) : auth(rpcAuth), conf(config)
+{
+	hc.reset(new HttpClient());
+	method = RpcAuth::ParseMethod(conf->getKmsMethod());
+}
+
+/**
+ * Set HttpClient object.
+ */
+void KmsClientProvider::setHttpClient(std::shared_ptr<HttpClient> hc)
+{
+	this->hc = hc;
+}
+
+/**
+ * Parse kms url from configure file.
+ */
+std::string KmsClientProvider::parseKmsUrl() 
+{
+	std::string start = "kms://";
+    std::string http = "http@";
+    std::string https = "https@";
+	std::string urlParse = conf->getKmsUrl();
+	LOG(DEBUG2, "KmsClientProvider : Get kms url from conf : %s.", urlParse.c_str());
+    if (urlParse.compare(0, start.length(), start) == 0) {
+        start = urlParse.substr(start.length());
+        if (start.compare(0, http.length(), http) == 0) {
+            return "http://" + start.substr(http.length());
+        }
+        else if (start.compare(0, https.length(), https) == 0) {
+            return "https://" + start.substr(https.length());
+        }
+        else
+            THROW(HdfsIOException, "Bad KMS provider URL: %s", urlParse.c_str());
+    }
+    else
+        THROW(HdfsIOException, "Bad KMS provider URL: %s", urlParse.c_str());
+
+}
+
+/**
+ * Build kms url based on urlSuffix and different auth method. 
+ */
+std::string KmsClientProvider::buildKmsUrl(const std::string &url, const std::string &urlSuffix)
+{
+		std::string baseUrl = url;
+        baseUrl = url + "/v1/" + urlSuffix;
+		std::size_t found = urlSuffix.find('?');
+
+        if (method == AuthMethod::KERBEROS) {
+            // todo
+			THROW(InvalidParameter, "KmsClientProvider : Not support kerberos yet.");
+        } else if (method == AuthMethod::SIMPLE) {
+            std::string user = auth->getUser().getRealUser();
+			LOG(DEBUG1, "KmsClientProvider : Kms urlSuffix is : %s. Auth real user is : %s.", urlSuffix.c_str(), user.c_str());
+            if (user.length() == 0)
+                user = auth->getUser().getKrbName();
+			if (found != std::string::npos)
+            	return baseUrl + "&user.name=" + user;
+			else
+				return baseUrl + "?user.name=" + user;
+        } else {
+            return baseUrl;
+        }	
+}
+
+/**
+ * Set common headers for kms API.
+ */
+void KmsClientProvider::setCommonHeaders(std::vector<std::string>& headers)
+{
+	headers.push_back("Content-Type: application/json");
+ 	headers.push_back("Accept: *");
+}
+
+
+/**
+ * Create an encryption key from kms.
+ * @param keyName the name of this key.
+ * @param cipher the ciphertext of this key. e.g. "AES/CTR/NoPadding" .
+ * @param length the length of this key.
+ * @param material will be encode to base64.
+ * @param description key's info.
+ */
+void KmsClientProvider::createKey(const std::string &keyName, const std::string &cipher, const int length, const std::string &material, const std::string &description)
+{
+	hc->init();
+	/* Prepare url for HttpClient.*/
+	url = parseKmsUrl(); 
+	std::string urlSuffix = "keys";
+	url = buildKmsUrl(url, urlSuffix);
+	/* Prepare headers for HttpClient.*/
+	std::vector<std::string> headers;
+	setCommonHeaders(headers);
+	/* Prepare body for HttpClient. */
+	ptree map;
+    map.put("name", keyName);
+    map.put("cipher", cipher);
+	map.put("description", description);
+    std::string body = toJson(map);	
+	/* Set options for HttpClient to get response. */
+	hc->setURL(url);
+	hc->setHeaders(headers);
+	hc->setBody(body);
+	hc->setRequestRetryTimes(conf->getHttpRequestRetryTimes());
+	hc->setRequestTimeout(conf->getCurlTimeOut());
+	hc->setExpectedResponseCode(201);
+	std::string response = hc->post();
+
+	LOG(INFO, "KmsClientProvider::createKey : The key name, key cipher, key length, key material, description are : %s, %s, %s, %s, %s. The kms url is : %s . The kms body is : %s. The response of kms server is : %s ." , keyName.c_str(), cipher.c_str(), length, material.c_str(), description.c_str(), url.c_str(), body.c_str(), response.c_str());
+		
+} 
+
+/**
+ * Get key metadata based on encrypted file's key name. 
+ * @param encryptionInfo the encryption info of file.
+ * @return return response info about key metadata from kms server.
+ */
+ptree KmsClientProvider::getKeyMetadata(const FileEncryptionInfo &encryptionInfo)
+{
+	hc->init();
+	url = parseKmsUrl(); 
+	std::string urlSuffix = "key/" + hc->escape(encryptionInfo.getKeyName()) + "/_metadata";
+	url = buildKmsUrl(url, urlSuffix);
+	
+	hc->setURL(url);
+	hc->setExpectedResponseCode(200);
+	hc->setRequestRetryTimes(conf->getHttpRequestRetryTimes());
+	hc->setRequestTimeout(conf->getCurlTimeOut());
+	std::string response = hc->get();
+	
+	LOG(INFO, "KmsClientProvider::getKeyMetadata : The kms url is : %s. The response of kms server is : %s ." , url.c_str(), response.c_str());
+
+	return fromJson(response);
+
+}
+
+/**
+ * Delete an encryption key from kms. 
+ * @param encryptionInfo the encryption info of file.
+ */
+void KmsClientProvider::deleteKey(const FileEncryptionInfo &encryptionInfo)
+{
+	hc->init();
+	url = parseKmsUrl(); 
+	std::string urlSuffix = "key/" + hc->escape(encryptionInfo.getKeyName());
+	url = buildKmsUrl(url, urlSuffix);
+	
+    hc->setURL(url);
+	hc->setExpectedResponseCode(200);
+	hc->setRequestRetryTimes(conf->getHttpRequestRetryTimes());
+	hc->setRequestTimeout(conf->getCurlTimeOut());
+	std::string response = hc->del();
+
+	LOG(INFO, "KmsClientProvider::deleteKey : The kms url is : %s. The response of kms server is : %s ." , url.c_str(), response.c_str());
+}
+
+/**
+ * Decrypt an encrypted key from kms.
+ * @param encryptionInfo the encryption info of file.
+ * @return return decrypted key.
+ */
+ptree KmsClientProvider::decryptEncryptedKey(const FileEncryptionInfo &encryptionInfo)
+{
+	hc->init();
+	/* Prepare HttpClient url. */
+	url = parseKmsUrl(); 
+	std::string urlSuffix = "keyversion/" + hc->escape(encryptionInfo.getEzKeyVersionName()) + "/_eek?eek_op=decrypt";
+	url = buildKmsUrl(url, urlSuffix);
+	/* Prepare HttpClient headers. */
+	std::vector<std::string> headers;
+	setCommonHeaders(headers);
+	/* Prepare HttpClient body with json format. */
+	ptree map;
+    map.put("name", encryptionInfo.getKeyName());
+    map.put("iv", base64Encode(encryptionInfo.getIv()));
+    map.put("material", base64Encode(encryptionInfo.getKey()));
+    std::string body = toJson(map);	
+
+	/* Set options for HttpClient. */
+	hc->setURL(url);
+	hc->setHeaders(headers);
+	hc->setBody(body);
+	hc->setExpectedResponseCode(200);
+	hc->setRequestRetryTimes(conf->getHttpRequestRetryTimes());
+	hc->setRequestTimeout(conf->getCurlTimeOut());
+	std::string response = hc->post();
+
+	LOG(INFO, "KmsClientProvider::decryptEncryptedKey : The kms url is : %s . The kms body is : %s. The response of kms server is : %s ." , url.c_str(), body.c_str(), response.c_str());
+	return fromJson(response);
+}
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c024842f/depends/libhdfs3/src/client/KmsClientProvider.h
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/src/client/KmsClientProvider.h b/depends/libhdfs3/src/client/KmsClientProvider.h
new file mode 100644
index 0000000..5eef428
--- /dev/null
+++ b/depends/libhdfs3/src/client/KmsClientProvider.h
@@ -0,0 +1,142 @@
+/********************************************************************
+ * 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_KMSCLIENTPROVIDER_H_
+#define _HDFS_LIBHDFS3_CLIENT_KMSCLIENTPROVIDER_H_
+
+#include <string>
+#include <gsasl.h>
+
+#include "openssl/conf.h"
+#include "openssl/evp.h"
+#include "openssl/err.h"
+#include "FileEncryptionInfo.h"
+#include "HttpClient.h"
+#include <vector>
+#include "common/SessionConfig.h"
+#include "rpc/RpcAuth.h"
+
+#include <boost/property_tree/ptree.hpp>
+
+using boost::property_tree::ptree;
+using namespace Hdfs::Internal;
+
+
+namespace Hdfs {
+
+class KmsClientProvider {
+public:
+
+	/**
+     * Construct a KmsClientProvider instance.
+     * @param auth RpcAuth to get the auth method and user info.
+	 * @param conf a SessionConfig to get the configuration.
+     */
+	KmsClientProvider(std::shared_ptr<RpcAuth> auth, std::shared_ptr<SessionConfig> conf);
+
+	/**
+     * Destroy a KmsClientProvider instance.
+     */
+	virtual ~KmsClientProvider(){
+	}
+
+	/**
+	 * Set HttpClient object.
+	 */
+	void setHttpClient(std::shared_ptr<HttpClient> hc);
+
+	/**
+	 * Create an encryption key from kms.
+	 * @param keyName the name of this key.
+	 * @param cipher the ciphertext of this key. e.g. "AES/CTR/NoPadding" .
+	 * @param length the length of this key.
+	 * @param material will be encode to base64.
+	 * @param description key's info.
+ 	*/	
+	virtual void createKey(const std::string &keyName, const std::string &cipher, const int length, const std::string &material, const std::string &description);
+
+	/**
+	 * Get key metadata based on encrypted file's key name.
+	 * @param encryptionInfo the encryption info of file.
+ 	* @return return response info about key metadata from kms server.
+	 */
+	virtual ptree getKeyMetadata(const FileEncryptionInfo &encryptionInfo);
+
+	/**
+ 	 * Delete an encryption key from kms.
+	 * @param encryptionInfo the encryption info of file.
+	 */
+	virtual void deleteKey(const FileEncryptionInfo &encryptionInfo);
+
+	/**
+ 	 * Decrypt an encrypted key from kms.
+	 * @param encryptionInfo the encryption info of file.
+	 * @return return decrypted key.
+	 */
+	virtual ptree decryptEncryptedKey(const FileEncryptionInfo &encryptionInfo);
+
+	/**
+	 * Encode string to base64.
+	 */
+	static std::string	base64Encode(const std::string &data);
+
+	/**
+	 * Decode base64 to string.
+	 */	
+	static std::string	base64Decode(const std::string &data);
+
+private:
+
+	/**
+	 * Convert ptree format to json format.
+	 */
+	static std::string  toJson(const ptree &data);
+
+	/**
+ 	 * Convert json format to ptree format.
+	 */
+	static ptree		fromJson(const std::string &data);
+
+	/**
+	 * Parse kms url from configure file.
+	 */
+	std::string 		parseKmsUrl();
+
+	/**
+	 * Build kms url based on urlSuffix and different auth method.
+	 */
+	std::string 		buildKmsUrl(const std::string &url, const std::string &urlSuffix);
+	/**
+ 	 * Set common headers for kms API.
+ 	 */
+	void setCommonHeaders(std::vector<std::string>& headers);	
+
+	std::shared_ptr<HttpClient> 	hc;
+	std::string						url;
+
+	std::shared_ptr<RpcAuth> 		auth;
+	AuthMethod						method;
+	std::shared_ptr<SessionConfig> 	conf;
+	
+};
+
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/c024842f/depends/libhdfs3/test/function/TestKmsClient.cpp
----------------------------------------------------------------------
diff --git a/depends/libhdfs3/test/function/TestKmsClient.cpp b/depends/libhdfs3/test/function/TestKmsClient.cpp
new file mode 100644
index 0000000..0f9bbc2
--- /dev/null
+++ b/depends/libhdfs3/test/function/TestKmsClient.cpp
@@ -0,0 +1,206 @@
+/********************************************************************
+ * 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 "client/FileSystem.h"
+#include "client/FileSystemInter.h"
+#include "DateTime.h"
+#include "Exception.h"
+#include "ExceptionInternal.h"
+#include "gtest/gtest.h"
+#include "TestUtil.h"
+#include "Thread.h"
+#include "XmlConfig.h"
+#include "client/KmsClientProvider.h"
+#include "client/HttpClient.h"
+#include "client/hdfs.h"
+
+#include <ctime>
+
+#ifndef TEST_HDFS_PREFIX
+#define TEST_HDFS_PREFIX "./"
+#endif
+
+#define BASE_DIR TEST_HDFS_PREFIX"/testKmsClient/"
+
+using namespace Hdfs;
+using namespace Hdfs::Internal;
+
+class TestKmsClient: public ::testing::Test {
+public:
+    TestKmsClient() : conf("function-test.xml") {
+        conf.set("hadoop.kms.authentication.type", "simple");
+		conf.set("dfs.encryption.key.provider.uri","kms://http@localhost:16000/kms");
+		sconf.reset(new SessionConfig(conf));	
+		userInfo.setRealUser("abai");
+		auth.reset(new RpcAuth(userInfo, RpcAuth::ParseMethod(sconf->getKmsMethod())));
+		hc.reset(new HttpClient());
+		kcp.reset(new KmsClientProvider(auth, sconf));
+		kcp->setHttpClient(hc);
+		fs.reset(new FileSystem(conf));
+		fs->connect();
+    }
+
+    ~TestKmsClient() {
+		try {
+            fs->disconnect();
+        } catch (...) {
+        }
+	}
+protected:
+    Config 							conf;
+	UserInfo			 			userInfo;
+	shared_ptr<RpcAuth> 			auth;
+	shared_ptr<HttpClient> 			hc;
+	shared_ptr<KmsClientProvider> 	kcp;
+	shared_ptr<SessionConfig>		sconf;
+	shared_ptr<FileSystem> 			fs;
+};
+
+static bool CreateFile(hdfsFS fs, const char * path, int64_t blockSize,
+                       int64_t fileSize) {
+    hdfsFile out;
+    size_t offset = 0;
+    int64_t todo = fileSize, batch;
+    std::vector<char> buffer(32 * 1024);
+    int rc = -1;
+
+    do {
+        if (NULL == (out = hdfsOpenFile(fs, path, O_WRONLY, 0, 0, blockSize))) {
+            break;
+        }
+
+        while (todo > 0) {
+            batch = todo < static_cast<int32_t>(buffer.size()) ?
+                    todo : buffer.size();
+            Hdfs::FillBuffer(&buffer[0], batch, offset);
+
+            if (0 > (rc = hdfsWrite(fs, out, &buffer[0], batch))) {
+                break;
+            }
+
+            todo -= rc;
+            offset += rc;
+        }
+
+        rc = hdfsCloseFile(fs, out);
+    } while (0);
+
+    return rc >= 0;
+}
+
+
+TEST_F(TestKmsClient, CreateKeySuccess) {
+	std::string keyName = "testcreatekeyname";
+	std::string cipher = "AES/CTR/NoPadding";
+	int length = 128;
+	std::string material = "testCreateKey";
+	std::string description = "Test create key success.";
+	ASSERT_NO_THROW(kcp->createKey(keyName, cipher, length, material, description));
+}
+
+TEST_F(TestKmsClient, GetKeyMetadataSuccess) {
+	FileEncryptionInfo encryptionInfo;
+    encryptionInfo.setKeyName("testcreatekeyname");
+	ptree map = kcp->getKeyMetadata(encryptionInfo);
+	std::string keyName = map.get<std::string>("name");
+	ASSERT_STREQ("testcreatekeyname", keyName.c_str());
+}
+
+TEST_F(TestKmsClient, DeleteKeySuccess) {
+	FileEncryptionInfo encryptionInfo;
+    encryptionInfo.setKeyName("testcreatekeyname");
+	ASSERT_NO_THROW(kcp->deleteKey(encryptionInfo));
+}
+
+
+TEST_F(TestKmsClient, DecryptEncryptedKeySuccess) {
+	hdfsFS hfs = NULL;
+    struct hdfsBuilder * bld = hdfsNewBuilder();
+    assert(bld != NULL);
+    hdfsBuilderSetNameNode(bld, "default");
+    hfs = hdfsBuilderConnect(bld);
+	
+	//create key
+	hc.reset(new HttpClient());
+	kcp.reset(new KmsClientProvider(auth, sconf));
+	kcp->setHttpClient(hc);
+	std::string keyName = "testdekeyname";
+    std::string cipher = "AES/CTR/NoPadding";
+    int length = 128;
+    std::string material = "test DEK";
+    std::string description = "Test DEK create key success.";
+	kcp->createKey(keyName, cipher, length, material, description);
+	
+	//delete dir
+	EXPECT_EQ(0, hdfsDelete(hfs, BASE_DIR"/testDEKey", true));
+	
+	//create dir
+	EXPECT_EQ(0, hdfsCreateDirectory(hfs, BASE_DIR"/testDEKey"));
+
+	//create encryption zone and encrypted file	
+	ASSERT_EQ(0, hdfsCreateEncryptionZone(hfs, BASE_DIR"/testDEKey", "testdekeyname"));
+	const char * tdeFile = BASE_DIR"/testDEKey/tdefile";
+	ASSERT_TRUE(CreateFile(hfs, tdeFile, 0, 0));
+
+	//decrypt encrypted key
+	hc.reset(new HttpClient());
+	kcp.reset(new KmsClientProvider(auth, sconf));
+	kcp->setHttpClient(hc);
+	FileStatus fileStatus = fs->getFileStatus(tdeFile);
+	FileEncryptionInfo *enInfo = fileStatus.getFileEncryption();
+	ptree map = kcp->decryptEncryptedKey(*enInfo);
+	std::string versionName = map.get<std::string>("versionName");
+	ASSERT_STREQ("EK", versionName.c_str());
+
+	//delete key
+	hc.reset(new HttpClient());
+	kcp.reset(new KmsClientProvider(auth, sconf));
+	kcp->setHttpClient(hc);
+	FileEncryptionInfo encryptionInfo;
+    encryptionInfo.setKeyName("testdekeyname");
+    kcp->deleteKey(encryptionInfo);
+
+}
+
+TEST_F(TestKmsClient, CreateKeyFailediBadUrl) {
+	std::string keyName = "testcreatekeyfailname";
+    std::string cipher = "AES/CTR/NoPadding";
+    int length = 128;
+    std::string material = "testCreateKey";
+	
+	std::string url[4] = {
+		"ftp:///http@localhost:16000/kms",
+		"kms://htttp@localhost:16000/kms",
+		"kms:///httpss@localhost:16000/kms",
+		"kms:///http@localhost:16000/kms"	
+	};
+	for(int i=0; i<4; i++) {
+ 		conf.set("hadoop.kms.authentication.type", "simple");
+   		conf.set("dfs.encryption.key.provider.uri", url[i]);
+		sconf.reset(new SessionConfig(conf));	
+		userInfo.setRealUser("abai");
+		auth.reset(new RpcAuth(userInfo, RpcAuth::ParseMethod(sconf->getKmsMethod())));
+		hc.reset(new HttpClient());
+		ASSERT_THROW(kcp.reset(new KmsClientProvider(auth, sconf)), HdfsIOException);
+	}
+}
+
+