You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by bo...@apache.org on 2016/06/21 20:43:02 UTC

[1/3] hadoop git commit: HDFS-10511: libhdfs++: make error returning mechanism consistent across all hdfs operations. Contributed by Anatoli Shein.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-8707 71af40868 -> adb1a63e1


HDFS-10511: libhdfs++: make error returning mechanism consistent across all hdfs operations.  Contributed by Anatoli Shein.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5166a309
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5166a309
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5166a309

Branch: refs/heads/HDFS-8707
Commit: 5166a309768084cf036a314f70985ba932d4b7c5
Parents: 71af408
Author: Bob Hansen <bo...@hpe.com>
Authored: Tue Jun 21 16:26:58 2016 -0400
Committer: Bob Hansen <bo...@hpe.com>
Committed: Tue Jun 21 16:26:58 2016 -0400

----------------------------------------------------------------------
 .../native/libhdfspp/include/hdfspp/hdfs_ext.h  | 48 +++++++-------
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     | 67 ++++++++++++++++----
 .../src/main/native/libhdfspp/tests/hdfs_shim.c |  2 +-
 3 files changed, 79 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5166a309/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h
index 47ef792..af7393f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h
@@ -55,16 +55,17 @@
 #ifdef __cplusplus
 extern "C" {
 #endif
+
 /**
  *  Reads the last error, if any, that happened in this thread
  *  into the user supplied buffer.
  *  @param buf  A chunk of memory with room for the error string.
  *  @param len  Size of the buffer, if the message is longer than
  *              len len-1 bytes of the message will be copied.
+ *  @return     0 on successful read of the last error, -1 otherwise.
  **/
-
 LIBHDFS_EXTERNAL
-void hdfsGetLastError(char *buf, int len);
+int hdfsGetLastError(char *buf, int len);
 
 
 /**
@@ -94,7 +95,7 @@ struct hdfsBuilder *hdfsNewBuilderFromDirectory(const char * configDirectory);
  *                 key isn't found.  You must free this string with
  *                 hdfsConfStrFree.
  *
- * @return         0 on success; nonzero error code otherwise.
+ * @return         0 on success; -1 otherwise.
  *                 Failure to find the key is not an error.
  */
 LIBHDFS_EXTERNAL
@@ -108,28 +109,12 @@ int hdfsBuilderConfGetStr(struct hdfsBuilder *bld, const char *key,
  * @param val      (out param) The value.  This will NOT be changed if the
  *                 key isn't found.
  *
- * @return         0 on success; nonzero error code otherwise.
+ * @return         0 on success; -1 otherwise.
  *                 Failure to find the key is not an error.
  */
 LIBHDFS_EXTERNAL
 int hdfsBuilderConfGetInt(struct hdfsBuilder *bld, const char *key, int32_t *val);
 
-
-/**
- * Returns the block information and data nodes associated with a particular file.
- *
- * The hdfsBlockLocations structure will have zero or more hdfsBlockInfo elements,
- * which will have zero or more ip_addr elements indicating which datanodes have
- * each block.
- *
- * @param fs         A connected hdfs instance
- * @param path       Path of the file to query
- * @param locations  The address of an output pointer to contain the block information.
- *                   On success, this pointer must be later freed with hdfsFreeBlockLocations.
- *
- * @return         0 on success; nonzero error code otherwise.
- *                 If the file does not exist, an error will be returned.
- */
 struct hdfsDNInfo {
   const char *    ip_address;
   const char *    hostname;
@@ -157,6 +142,21 @@ struct hdfsBlockLocations
     struct hdfsBlockInfo * blocks;
 };
 
+/**
+ * Returns the block information and data nodes associated with a particular file.
+ *
+ * The hdfsBlockLocations structure will have zero or more hdfsBlockInfo elements,
+ * which will have zero or more ip_addr elements indicating which datanodes have
+ * each block.
+ *
+ * @param fs         A connected hdfs instance
+ * @param path       Path of the file to query
+ * @param locations  The address of an output pointer to contain the block information.
+ *                   On success, this pointer must be later freed with hdfsFreeBlockLocations.
+ *
+ * @return         0 on success; -1 otherwise.
+ *                 If the file does not exist, -1 will be returned and errno will be set.
+ */
 LIBHDFS_EXTERNAL
 int hdfsGetBlockLocations(hdfsFS fs, const char *path, struct hdfsBlockLocations ** locations);
 
@@ -164,7 +164,7 @@ int hdfsGetBlockLocations(hdfsFS fs, const char *path, struct hdfsBlockLocations
  * Frees up an hdfsBlockLocations pointer allocated by hdfsGetBlockLocations.
  *
  * @param locations    The previously-populated pointer allocated by hdfsGetBlockLocations
- * @return             0 on success, nonzero on error
+ * @return             0 on success, -1 on error
  */
 LIBHDFS_EXTERNAL
 int hdfsFreeBlockLocations(struct hdfsBlockLocations * locations);
@@ -200,21 +200,21 @@ void hdfsFreeLogData(LogData*);
 
 /**
  * Enable loggind functionality for a component.
- * Return 1 on failure, 0 otherwise.
+ * Return -1 on failure, 0 otherwise.
  **/
 LIBHDFS_EXTERNAL
 int hdfsEnableLoggingForComponent(int component);
 
 /**
  * Disable logging functionality for a component.
- * Return 1 on failure, 0 otherwise.
+ * Return -1 on failure, 0 otherwise.
  **/
 LIBHDFS_EXTERNAL
 int hdfsDisableLoggingForComponent(int component);
 
 /**
  * Set level between trace and error.
- * Return 1 on failure, 0 otherwise.
+ * Return -1 on failure, 0 otherwise.
  **/
 LIBHDFS_EXTERNAL
 int hdfsSetLoggingLevel(int component);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5166a309/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
index 458d56e..6c9a00d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
@@ -69,9 +69,15 @@ struct hdfsFile_internal {
 thread_local std::string errstr;
 
 /* Fetch last error that happened in this thread */
-void hdfsGetLastError(char *buf, int len) {
+int hdfsGetLastError(char *buf, int len) {
+  //No error message
+  if(errstr.empty()){
+    return -1;
+  }
+
+  //There is an error, but no room for the error message to be copied to
   if(nullptr == buf || len < 1) {
-    return;
+    return -1;
   }
 
   /* leave space for a trailing null */
@@ -84,6 +90,8 @@ void hdfsGetLastError(char *buf, int len) {
 
   /* stick in null */
   buf[copylen] = 0;
+
+  return 0;
 }
 
 /* Event callbacks for next open calls */
@@ -214,6 +222,7 @@ int hdfsFileIsOpenForRead(hdfsFile file) {
 hdfsFS doHdfsConnect(optional<std::string> nn, optional<tPort> port, optional<std::string> user, const Options & options) {
   try
   {
+    errno = 0;
     IoService * io_service = IoService::New();
 
     FileSystem *fs = FileSystem::New(io_service, user.value_or(""), options);
@@ -270,6 +279,7 @@ hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user) {
 int hdfsDisconnect(hdfsFS fs) {
   try
   {
+    errno = 0;
     if (!fs) {
       ReportError(ENODEV, "Cannot disconnect null FS handle.");
       return -1;
@@ -288,6 +298,7 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, int bufferSize,
                       short replication, tSize blocksize) {
   try
   {
+    errno = 0;
     (void)flags;
     (void)bufferSize;
     (void)replication;
@@ -315,6 +326,7 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char *path, int flags, int bufferSize,
 int hdfsCloseFile(hdfsFS fs, hdfsFile file) {
   try
   {
+    errno = 0;
     if (!CheckSystemAndHandle(fs, file)) {
       return -1;
     }
@@ -424,6 +436,7 @@ void StatInfoToHdfsFileInfo(hdfsFileInfo * file_info,
 
 hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path) {
   try {
+    errno = 0;
     if (!CheckSystem(fs)) {
        return nullptr;
     }
@@ -448,6 +461,7 @@ hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path) {
 
 hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries) {
   try {
+      errno = 0;
       if (!CheckSystem(fs)) {
         *numEntries = 0;
         return nullptr;
@@ -485,6 +499,7 @@ hdfsFileInfo *hdfsListDirectory(hdfsFS fs, const char* path, int *numEntries) {
 
 void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
 {
+    errno = 0;
     int i;
     for (i = 0; i < numEntries; ++i) {
         delete[] hdfsFileInfo[i].mName;
@@ -593,6 +608,7 @@ tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void *buffer,
                 tSize length) {
   try
   {
+    errno = 0;
     if (!CheckSystemAndHandle(fs, file)) {
       return -1;
     }
@@ -613,9 +629,10 @@ tSize hdfsPread(hdfsFS fs, hdfsFile file, tOffset position, void *buffer,
 tSize hdfsRead(hdfsFS fs, hdfsFile file, void *buffer, tSize length) {
   try
   {
-  if (!CheckSystemAndHandle(fs, file)) {
-    return -1;
-  }
+    errno = 0;
+    if (!CheckSystemAndHandle(fs, file)) {
+      return -1;
+    }
 
     size_t len = length;
     Status stat = file->get_impl()->Read(buffer, &len);
@@ -635,6 +652,7 @@ tSize hdfsRead(hdfsFS fs, hdfsFile file, void *buffer, tSize length) {
 int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos) {
   try
   {
+    errno = 0;
     if (!CheckSystemAndHandle(fs, file)) {
       return -1;
     }
@@ -656,6 +674,7 @@ int hdfsSeek(hdfsFS fs, hdfsFile file, tOffset desiredPos) {
 tOffset hdfsTell(hdfsFS fs, hdfsFile file) {
   try
   {
+    errno = 0;
     if (!CheckSystemAndHandle(fs, file)) {
       return -1;
     }
@@ -678,6 +697,7 @@ tOffset hdfsTell(hdfsFS fs, hdfsFile file) {
 int hdfsCancel(hdfsFS fs, hdfsFile file) {
   try
   {
+    errno = 0;
     if (!CheckSystemAndHandle(fs, file)) {
       return -1;
     }
@@ -695,12 +715,13 @@ int hdfsGetBlockLocations(hdfsFS fs, const char *path, struct hdfsBlockLocations
 {
   try
   {
+    errno = 0;
     if (!CheckSystem(fs)) {
       return -1;
     }
     if (locations_out == nullptr) {
       ReportError(EINVAL, "Null pointer passed to hdfsGetBlockLocations");
-      return -2;
+      return -1;
     }
 
     std::shared_ptr<FileBlockLocation> ppLocations;
@@ -759,6 +780,7 @@ int hdfsGetBlockLocations(hdfsFS fs, const char *path, struct hdfsBlockLocations
 }
 
 int hdfsFreeBlockLocations(struct hdfsBlockLocations * blockLocations) {
+  errno = 0;
   if (blockLocations == nullptr)
     return 0;
 
@@ -861,6 +883,7 @@ HdfsConfiguration LoadDefault(ConfigurationLoader & loader)
 
 hdfsBuilder::hdfsBuilder() : config(loader.New<HdfsConfiguration>())
 {
+  errno = 0;
   loader.SetDefaultSearchPath();
   config = LoadDefault(loader);
 }
@@ -868,6 +891,7 @@ hdfsBuilder::hdfsBuilder() : config(loader.New<HdfsConfiguration>())
 hdfsBuilder::hdfsBuilder(const char * directory) :
       config(loader.New<HdfsConfiguration>())
 {
+  errno = 0;
   loader.SetSearchPath(directory);
   config = LoadDefault(loader);
 }
@@ -876,6 +900,7 @@ struct hdfsBuilder *hdfsNewBuilder(void)
 {
   try
   {
+    errno = 0;
     return new struct hdfsBuilder();
   } catch (const std::exception & e) {
     ReportException(e);
@@ -888,16 +913,19 @@ struct hdfsBuilder *hdfsNewBuilder(void)
 
 void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn)
 {
+  errno = 0;
   bld->overrideHost = std::string(nn);
 }
 
 void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port)
 {
+  errno = 0;
   bld->overridePort = port;
 }
 
 void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName)
 {
+  errno = 0;
   if (userName && *userName) {
     bld->user = std::string(userName);
   }
@@ -908,6 +936,7 @@ void hdfsFreeBuilder(struct hdfsBuilder *bld)
 {
   try
   {
+    errno = 0;
     delete bld;
   } catch (const std::exception & e) {
     ReportException(e);
@@ -921,6 +950,7 @@ int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
 {
   try
   {
+    errno = 0;
     optional<HdfsConfiguration> newConfig = bld->loader.OverlayValue(bld->config, key, val);
     if (newConfig)
     {
@@ -930,7 +960,7 @@ int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
     else
     {
       ReportError(EINVAL, "Could not change Builder value");
-      return 1;
+      return -1;
     }
   } catch (const std::exception & e) {
     return ReportException(e);
@@ -941,6 +971,7 @@ int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
 
 void hdfsConfStrFree(char *val)
 {
+  errno = 0;
   free(val);
 }
 
@@ -952,6 +983,7 @@ int hdfsConfGetStr(const char *key, char **val)
 {
   try
   {
+    errno = 0;
     hdfsBuilder builder;
     return hdfsBuilderConfGetStr(&builder, key, val);
   } catch (const std::exception & e) {
@@ -965,6 +997,7 @@ int hdfsConfGetInt(const char *key, int32_t *val)
 {
   try
   {
+    errno = 0;
     hdfsBuilder builder;
     return hdfsBuilderConfGetInt(&builder, key, val);
   } catch (const std::exception & e) {
@@ -981,6 +1014,7 @@ struct hdfsBuilder *hdfsNewBuilderFromDirectory(const char * configDirectory)
 {
   try
   {
+    errno = 0;
     return new struct hdfsBuilder(configDirectory);
   } catch (const std::exception & e) {
     ReportException(e);
@@ -996,6 +1030,7 @@ int hdfsBuilderConfGetStr(struct hdfsBuilder *bld, const char *key,
 {
   try
   {
+    errno = 0;
     optional<std::string> value = bld->config.Get(key);
     if (value)
     {
@@ -1027,14 +1062,17 @@ int hdfsBuilderConfGetInt(struct hdfsBuilder *bld, const char *key, int32_t *val
 {
   try
   {
+    errno = 0;
     // Pull from default configuration
     optional<int64_t> value = bld->config.GetInt(key);
     if (value)
     {
-      if (!isValidInt(*value))
-        return 1;
-
+      if (!isValidInt(*value)){
+        ReportError(EINVAL, "Builder value is not valid");
+        return -1;
+      }
       *val = *value;
+      return 0;
     }
     // If not found, don't change val
     ReportError(EINVAL, "Could not get Builder value");
@@ -1160,22 +1198,25 @@ static bool IsComponentValid(int component) {
 }
 
 int hdfsEnableLoggingForComponent(int component) {
+  errno = 0;
   if(!IsComponentValid(component))
-    return 1;
+    return -1;
   LogManager::EnableLogForComponent(static_cast<LogSourceComponent>(component));
   return 0;
 }
 
 int hdfsDisableLoggingForComponent(int component) {
+  errno = 0;
   if(!IsComponentValid(component))
-    return 1;
+    return -1;
   LogManager::DisableLogForComponent(static_cast<LogSourceComponent>(component));
   return 0;
 }
 
 int hdfsSetLoggingLevel(int level) {
+  errno = 0;
   if(!IsLevelValid(level))
-    return 1;
+    return -1;
   LogManager::SetLogLevel(static_cast<LogLevel>(level));
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5166a309/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
index 3438dc3..12e9f71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
@@ -388,7 +388,7 @@ void hadoopRzBufferFree(hdfsFile file, struct hadoopRzBuffer *buffer) {
  * hdfs_ext functions
  */
 
-void hdfsGetLastError(char *buf, int len) {
+int hdfsGetLastError(char *buf, int len) {
   return libhdfspp_hdfsGetLastError(buf, len);
 }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[2/3] hadoop git commit: HDFS-10515: libhdfs++: Implement mkdirs, rmdir, rename, and remove

Posted by bo...@apache.org.
HDFS-10515: libhdfs++: Implement mkdirs, rmdir, rename, and remove


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

Branch: refs/heads/HDFS-8707
Commit: a1c87ccbb5a21bd1b75b583ccdd525bf6a70a123
Parents: 5166a30
Author: Bob Hansen <bo...@hpe.com>
Authored: Tue Jun 21 16:27:33 2016 -0400
Committer: Bob Hansen <bo...@hpe.com>
Committed: Tue Jun 21 16:27:33 2016 -0400

----------------------------------------------------------------------
 .../libhdfs-tests/test_libhdfs_threaded.c       |  20 ++-
 .../native/libhdfspp/include/hdfspp/hdfspp.h    |  33 ++++
 .../native/libhdfspp/include/hdfspp/status.h    |   2 +
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     |  88 +++++++++-
 .../main/native/libhdfspp/lib/common/status.cc  |   6 +
 .../main/native/libhdfspp/lib/fs/filesystem.cc  | 175 ++++++++++++++-----
 .../main/native/libhdfspp/lib/fs/filesystem.h   |  11 ++
 .../libhdfspp/lib/fs/namenode_operations.cc     | 153 +++++++++++++++-
 .../libhdfspp/lib/fs/namenode_operations.h      |   9 +
 .../native/libhdfspp/tests/hdfs_ext_test.cc     |  77 ++++++++
 .../src/main/native/libhdfspp/tests/hdfs_shim.c |   6 +-
 11 files changed, 517 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
index 1f6cd3f..c3c2db8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
@@ -166,6 +166,7 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     if (numEntries != 0) {
         fprintf(stderr, "hdfsListDirectory set numEntries to "
                 "%d on empty directory.", numEntries);
+        return EIO;
     }
 
     /* There should not be any file to open for reading. */
@@ -204,21 +205,26 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     hdfsFreeFileInfo(dirList, numEntries);
 
     /* Create many files for ListDirectory to page through */
+    char listDirTest[PATH_MAX];
+    strcpy(listDirTest, paths->prefix);
+    strcat(listDirTest, "/for_list_test/");
+    EXPECT_ZERO(hdfsCreateDirectory(fs, listDirTest));
     int nFile;
     for (nFile = 0; nFile < 10000; nFile++) {
       char filename[PATH_MAX];
-      snprintf(filename, PATH_MAX, "%s/many_files_%d", paths->prefix, nFile);
+      snprintf(filename, PATH_MAX, "%s/many_files_%d", listDirTest, nFile);
       file = hdfsOpenFile(fs, filename, O_WRONLY, 0, 0, 0);
       EXPECT_NONNULL(file);
       EXPECT_ZERO(hdfsCloseFile(fs, file));
     }
-    dirList = hdfsListDirectory(fs, paths->prefix, &numEntries);
+    dirList = hdfsListDirectory(fs, listDirTest, &numEntries);
     EXPECT_NONNULL(dirList);
-    if (numEntries != 10002) {
+    hdfsFreeFileInfo(dirList, numEntries);
+    if (numEntries != 10000) {
         fprintf(stderr, "hdfsListDirectory set numEntries to "
-                "%d on directory containing 10002 files.", numEntries);
+                "%d on directory containing 10000 files.", numEntries);
+        return EIO;
     }
-    hdfsFreeFileInfo(dirList, numEntries);
 
     /* Let's re-open the file for reading */
     file = hdfsOpenFile(fs, paths->file1, O_RDONLY, 0, 0, 0);
@@ -253,8 +259,8 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     EXPECT_ZERO(memcmp(paths->prefix, tmp, expected));
     EXPECT_ZERO(hdfsCloseFile(fs, file));
 
-    // TODO: Non-recursive delete should fail?
-    //EXPECT_NONZERO(hdfsDelete(fs, prefix, 0));
+    //Non-recursive delete fails
+    EXPECT_NONZERO(hdfsDelete(fs, paths->prefix, 0));
     EXPECT_ZERO(hdfsCopy(fs, paths->file1, fs, paths->file2));
 
     EXPECT_ZERO(hdfsChown(fs, paths->file2, NULL, NULL));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
index 26e26a8..73a9de9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
@@ -217,6 +217,39 @@ class FileSystem {
   virtual Status GetBlockLocations(const std::string & path,
     std::shared_ptr<FileBlockLocation> * locations) = 0;
 
+  /**
+   * Creates a new directory
+   *
+   *  @param path           Path to the directory to be created (must be non-empty)
+   *  @param permissions    Permissions for the new directory   (negative value for the default permissions)
+   *  @param createparent   Create parent directories if they do not exist (may not be empty)
+   */
+  virtual void Mkdirs(const std::string & path, long permissions, bool createparent,
+      std::function<void(const Status &)> handler) = 0;
+  virtual Status Mkdirs(const std::string & path, long permissions, bool createparent) = 0;
+
+  /**
+   *  Delete the given file or directory from the file system.
+   *  <p>
+   *  same as delete but provides a way to avoid accidentally
+   *  deleting non empty directories programmatically.
+   *  @param path existing name (must be non-empty)
+   *  @param recursive if true deletes a non empty directory recursively
+   */
+  virtual void Delete(const std::string &path, bool recursive,
+      const std::function<void(const Status &)> &handler) = 0;
+  virtual Status Delete(const std::string &path, bool recursive) = 0;
+
+  /**
+   *  Rename - Rename file.
+   *  @param oldPath The path of the source file.       (must be non-empty)
+   *  @param newPath The path of the destination file.  (must be non-empty)
+   *  @return Returns 0 on success, -1 on error.
+   */
+  virtual void Rename(const std::string &oldPath, const std::string &newPath,
+      const std::function<void(const Status &)> &handler) = 0;
+  virtual Status Rename(const std::string &oldPath, const std::string &newPath) = 0;
+
   /*****************************************************************************
    *                    FILE SYSTEM SNAPSHOT FUNCTIONS
    ****************************************************************************/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
index 0877669..1628d8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
@@ -59,6 +59,8 @@ class Status {
     kPermissionDenied = static_cast<unsigned>(std::errc::permission_denied),
     kPathNotFound = static_cast<unsigned>(std::errc::no_such_file_or_directory),
     kNotADirectory = static_cast<unsigned>(std::errc::not_a_directory),
+    kFileAlreadyExists = static_cast<unsigned>(std::errc::file_exists),
+    kPathIsNotEmptyDirectory = static_cast<unsigned>(std::errc::directory_not_empty),
     kException = 256,
     kAuthenticationFailed = 257,
   };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
index 6c9a00d..deb7eb0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
@@ -165,6 +165,14 @@ static int Error(const Status &stat) {
       errnum = ENOTDIR;
       default_message = "Not a directory";
       break;
+    case Status::Code::kFileAlreadyExists:
+      errnum = EEXIST;
+      default_message = "File already exists";
+      break;
+    case Status::Code::kPathIsNotEmptyDirectory:
+      errnum = ENOTEMPTY;
+      default_message = "Directory is not empty";
+      break;
     default:
       errnum = ENOSYS;
       default_message = "Error: unrecognised code";
@@ -509,6 +517,76 @@ void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
     delete[] hdfsFileInfo;
 }
 
+int hdfsCreateDirectory(hdfsFS fs, const char* path) {
+  try {
+    errno = 0;
+    if (!CheckSystem(fs)) {
+      return -1;
+    }
+    if (!path) {
+      return Error(Status::InvalidArgument("hdfsCreateDirectory: argument 'path' cannot be NULL"));
+    }
+    Status stat;
+    //-1 for default permissions and true for creating all non-existant parent directories
+    stat = fs->get_impl()->Mkdirs(path, -1, true);
+    if (!stat.ok()) {
+      return Error(stat);
+    }
+    return 0;
+  } catch (const std::exception & e) {
+    return ReportException(e);
+  } catch (...) {
+    return ReportCaughtNonException();
+  }
+}
+
+int hdfsDelete(hdfsFS fs, const char* path, int recursive) {
+  try {
+      errno = 0;
+      if (!CheckSystem(fs)) {
+        return -1;
+      }
+      if (!path) {
+        return Error(Status::InvalidArgument("hdfsDelete: argument 'path' cannot be NULL"));
+      }
+      Status stat;
+      stat = fs->get_impl()->Delete(path, recursive);
+      if (!stat.ok()) {
+        return Error(stat);
+      }
+      return 0;
+    } catch (const std::exception & e) {
+      return ReportException(e);
+    } catch (...) {
+      return ReportCaughtNonException();
+    }
+}
+
+int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath) {
+  try {
+    errno = 0;
+    if (!CheckSystem(fs)) {
+      return -1;
+    }
+    if (!oldPath) {
+      return Error(Status::InvalidArgument("hdfsRename: argument 'oldPath' cannot be NULL"));
+    }
+    if (!newPath) {
+      return Error(Status::InvalidArgument("hdfsRename: argument 'newPath' cannot be NULL"));
+    }
+    Status stat;
+    stat = fs->get_impl()->Rename(oldPath, newPath);
+    if (!stat.ok()) {
+      return Error(stat);
+    }
+    return 0;
+  } catch (const std::exception & e) {
+    return ReportException(e);
+  } catch (...) {
+    return ReportCaughtNonException();
+  }
+}
+
 int hdfsCreateSnapshot(hdfsFS fs, const char* path, const char* name) {
   try {
     errno = 0;
@@ -516,7 +594,7 @@ int hdfsCreateSnapshot(hdfsFS fs, const char* path, const char* name) {
       return -1;
     }
     if (!path) {
-      return Error(Status::InvalidArgument("Argument 'path' cannot be NULL"));
+      return Error(Status::InvalidArgument("hdfsCreateSnapshot: argument 'path' cannot be NULL"));
     }
     Status stat;
     if(!name){
@@ -542,10 +620,10 @@ int hdfsDeleteSnapshot(hdfsFS fs, const char* path, const char* name) {
       return -1;
     }
     if (!path) {
-      return Error(Status::InvalidArgument("Argument 'path' cannot be NULL"));
+      return Error(Status::InvalidArgument("hdfsDeleteSnapshot: argument 'path' cannot be NULL"));
     }
     if (!name) {
-      return Error(Status::InvalidArgument("Argument 'name' cannot be NULL"));
+      return Error(Status::InvalidArgument("hdfsDeleteSnapshot: argument 'name' cannot be NULL"));
     }
     Status stat;
     stat = fs->get_impl()->DeleteSnapshot(path, name);
@@ -567,7 +645,7 @@ int hdfsAllowSnapshot(hdfsFS fs, const char* path) {
       return -1;
     }
     if (!path) {
-      return Error(Status::InvalidArgument("Argument 'path' cannot be NULL"));
+      return Error(Status::InvalidArgument("hdfsAllowSnapshot: argument 'path' cannot be NULL"));
     }
     Status stat;
     stat = fs->get_impl()->AllowSnapshot(path);
@@ -589,7 +667,7 @@ int hdfsDisallowSnapshot(hdfsFS fs, const char* path) {
       return -1;
     }
     if (!path) {
-      return Error(Status::InvalidArgument("Argument 'path' cannot be NULL"));
+      return Error(Status::InvalidArgument("hdfsDisallowSnapshot: argument 'path' cannot be NULL"));
     }
     Status stat;
     stat = fs->get_impl()->DisallowSnapshot(path);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
index a781d7b..d6c9875 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
@@ -30,6 +30,8 @@ const char * kPathNotFoundException = "org.apache.hadoop.fs.InvalidPathException
 const char * kPathNotFoundException2 = "java.io.FileNotFoundException";
 const char * kPathIsNotDirectoryException = "org.apache.hadoop.fs.PathIsNotDirectoryException";
 const char * kSnapshotException = "org.apache.hadoop.hdfs.protocol.SnapshotException";
+const char * kFileAlreadyExistsException = "org.apache.hadoop.fs.FileAlreadyExistsException";
+const char * kPathIsNotEmptyDirectoryException = "org.apache.hadoop.fs.PathIsNotEmptyDirectoryException";
 
 Status::Status(int code, const char *msg1) : code_(code) {
   if(msg1) {
@@ -82,6 +84,10 @@ Status Status::Exception(const char *exception_class_name, const char *error_mes
       return Status(kNotADirectory, error_message);
   else if (exception_class_name && (strcmp(exception_class_name, kSnapshotException) == 0))
         return Status(kInvalidArgument, error_message);
+  else if (exception_class_name && (strcmp(exception_class_name, kFileAlreadyExistsException) == 0))
+          return Status(kFileAlreadyExists, error_message);
+  else if (exception_class_name && (strcmp(exception_class_name, kPathIsNotEmptyDirectoryException) == 0))
+          return Status(kPathIsNotEmptyDirectory, error_message);
   else
     return Status(kException, exception_class_name, error_message);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
index 0648c13..1d11c21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
@@ -362,9 +362,7 @@ void FileSystemImpl::GetFileInfo(
                                  << FMT_THIS_ADDR << ", path="
                                  << path << ") called");
 
-  nn_.GetFileInfo(path, [handler](const Status &stat, const StatInfo &stat_info) {
-    handler(stat, stat_info);
-  });
+  nn_.GetFileInfo(path, handler);
 }
 
 Status FileSystemImpl::GetFileInfo(const std::string &path,
@@ -401,9 +399,7 @@ void FileSystemImpl::GetFsStats(
   LOG_DEBUG(kFileSystem,
       << "FileSystemImpl::GetFsStats(" << FMT_THIS_ADDR << ") called");
 
-  nn_.GetFsStats([handler](const Status &stat, const FsInfo &fs_info) {
-    handler(stat, fs_info);
-  });
+  nn_.GetFsStats(handler);
 }
 
 Status FileSystemImpl::GetFsStats(FsInfo & fs_info) {
@@ -512,6 +508,115 @@ Status FileSystemImpl::GetListing(const std::string &path, std::shared_ptr<std::
   return stat;
 }
 
+void FileSystemImpl::Mkdirs(const std::string & path, long permissions, bool createparent,
+    std::function<void(const Status &)> handler) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::Mkdirs(" << FMT_THIS_ADDR << ", path=" << path <<
+      ", permissions=" << permissions << ", createparent=" << createparent << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("Mkdirs: argument 'path' cannot be empty"));
+    return;
+  }
+
+  nn_.Mkdirs(path, permissions, createparent, handler);
+}
+
+Status FileSystemImpl::Mkdirs(const std::string & path, long permissions, bool createparent) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::[sync]Mkdirs(" << FMT_THIS_ADDR << ", path=" << path <<
+      ", permissions=" << permissions << ", createparent=" << createparent << ") called");
+
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
+
+  /* wrap async FileSystem::Mkdirs with promise to make it a blocking call */
+  auto h = [callstate](const Status &s) {
+    callstate->set_value(s);
+  };
+
+  Mkdirs(path, permissions, createparent, h);
+
+  /* block until promise is set */
+  auto returnstate = future.get();
+  Status stat = returnstate;
+
+  return stat;
+}
+
+void FileSystemImpl::Delete(const std::string &path, bool recursive,
+    const std::function<void(const Status &)> &handler) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::Delete(" << FMT_THIS_ADDR << ", path=" << path << ", recursive=" << recursive << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("Delete: argument 'path' cannot be empty"));
+    return;
+  }
+
+  nn_.Delete(path, recursive, handler);
+}
+
+Status FileSystemImpl::Delete(const std::string &path, bool recursive) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::[sync]Delete(" << FMT_THIS_ADDR << ", path=" << path << ", recursive=" << recursive << ") called");
+
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
+
+  /* wrap async FileSystem::Delete with promise to make it a blocking call */
+  auto h = [callstate](const Status &s) {
+    callstate->set_value(s);
+  };
+
+  Delete(path, recursive, h);
+
+  /* block until promise is set */
+  auto returnstate = future.get();
+  Status stat = returnstate;
+
+  return stat;
+}
+
+void FileSystemImpl::Rename(const std::string &oldPath, const std::string &newPath,
+    const std::function<void(const Status &)> &handler) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::Rename(" << FMT_THIS_ADDR << ", oldPath=" << oldPath << ", newPath=" << newPath << ") called");
+
+  if (oldPath.empty()) {
+    handler(Status::InvalidArgument("Rename: argument 'oldPath' cannot be empty"));
+    return;
+  }
+
+  if (newPath.empty()) {
+    handler(Status::InvalidArgument("Rename: argument 'newPath' cannot be empty"));
+    return;
+  }
+
+  nn_.Rename(oldPath, newPath, handler);
+}
+
+Status FileSystemImpl::Rename(const std::string &oldPath, const std::string &newPath) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::[sync]Rename(" << FMT_THIS_ADDR << ", oldPath=" << oldPath << ", newPath=" << newPath << ") called");
+
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
+
+  /* wrap async FileSystem::Rename with promise to make it a blocking call */
+  auto h = [callstate](const Status &s) {
+    callstate->set_value(s);
+  };
+
+  Rename(oldPath, newPath, h);
+
+  /* block until promise is set */
+  auto returnstate = future.get();
+  Status stat = returnstate;
+
+  return stat;
+}
+
 void FileSystemImpl::CreateSnapshot(const std::string &path,
     const std::string &name,
     const std::function<void(const Status &)> &handler) {
@@ -519,13 +624,11 @@ void FileSystemImpl::CreateSnapshot(const std::string &path,
       << "FileSystemImpl::CreateSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("CreateSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
-  nn_.CreateSnapshot(path, name, [handler](const Status &stat) {
-    handler(stat);
-  });
+  nn_.CreateSnapshot(path, name, handler);
 }
 
 Status FileSystemImpl::CreateSnapshot(const std::string &path,
@@ -533,19 +636,19 @@ Status FileSystemImpl::CreateSnapshot(const std::string &path,
   LOG_DEBUG(kFileSystem,
       << "FileSystemImpl::[sync]CreateSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
-  auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
-  std::future<std::tuple<Status>> future(callstate->get_future());
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
 
   /* wrap async FileSystem::CreateSnapshot with promise to make it a blocking call */
   auto h = [callstate](const Status &s) {
-    callstate->set_value(std::make_tuple(s));
+    callstate->set_value(s);
   };
 
   CreateSnapshot(path, name, h);
 
   /* block until promise is set */
   auto returnstate = future.get();
-  Status stat = std::get<0>(returnstate);
+  Status stat = returnstate;
 
   return stat;
 }
@@ -557,17 +660,15 @@ void FileSystemImpl::DeleteSnapshot(const std::string &path,
       << "FileSystemImpl::DeleteSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("DeleteSnapshot: argument 'path' cannot be empty"));
     return;
   }
   if (name.empty()) {
-    handler(Status::InvalidArgument("Argument 'name' cannot be empty"));
+    handler(Status::InvalidArgument("DeleteSnapshot: argument 'name' cannot be empty"));
     return;
   }
 
-  nn_.DeleteSnapshot(path, name, [handler](const Status &stat) {
-    handler(stat);
-  });
+  nn_.DeleteSnapshot(path, name, handler);
 }
 
 Status FileSystemImpl::DeleteSnapshot(const std::string &path,
@@ -575,19 +676,19 @@ Status FileSystemImpl::DeleteSnapshot(const std::string &path,
   LOG_DEBUG(kFileSystem,
       << "FileSystemImpl::[sync]DeleteSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
-  auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
-  std::future<std::tuple<Status>> future(callstate->get_future());
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
 
   /* wrap async FileSystem::DeleteSnapshot with promise to make it a blocking call */
   auto h = [callstate](const Status &s) {
-    callstate->set_value(std::make_tuple(s));
+    callstate->set_value(s);
   };
 
   DeleteSnapshot(path, name, h);
 
   /* block until promise is set */
   auto returnstate = future.get();
-  Status stat = std::get<0>(returnstate);
+  Status stat = returnstate;
 
   return stat;
 }
@@ -598,32 +699,30 @@ void FileSystemImpl::AllowSnapshot(const std::string &path,
       << "FileSystemImpl::AllowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("AllowSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
-  nn_.AllowSnapshot(path, [handler](const Status &stat) {
-    handler(stat);
-  });
+  nn_.AllowSnapshot(path, handler);
 }
 
 Status FileSystemImpl::AllowSnapshot(const std::string &path) {
   LOG_DEBUG(kFileSystem,
       << "FileSystemImpl::[sync]AllowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
-  auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
-  std::future<std::tuple<Status>> future(callstate->get_future());
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
 
   /* wrap async FileSystem::AllowSnapshot with promise to make it a blocking call */
   auto h = [callstate](const Status &s) {
-    callstate->set_value(std::make_tuple(s));
+    callstate->set_value(s);
   };
 
   AllowSnapshot(path, h);
 
   /* block until promise is set */
   auto returnstate = future.get();
-  Status stat = std::get<0>(returnstate);
+  Status stat = returnstate;
 
   return stat;
 }
@@ -634,32 +733,30 @@ void FileSystemImpl::DisallowSnapshot(const std::string &path,
       << "FileSystemImpl::DisallowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("DisallowSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
-  nn_.DisallowSnapshot(path, [handler](const Status &stat) {
-    handler(stat);
-  });
+  nn_.DisallowSnapshot(path, handler);
 }
 
 Status FileSystemImpl::DisallowSnapshot(const std::string &path) {
   LOG_DEBUG(kFileSystem,
       << "FileSystemImpl::[sync]DisallowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
-  auto callstate = std::make_shared<std::promise<std::tuple<Status>>>();
-  std::future<std::tuple<Status>> future(callstate->get_future());
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
 
   /* wrap async FileSystem::DisallowSnapshot with promise to make it a blocking call */
   auto h = [callstate](const Status &s) {
-    callstate->set_value(std::make_tuple(s));
+    callstate->set_value(s);
   };
 
   DisallowSnapshot(path, h);
 
   /* block until promise is set */
   auto returnstate = future.get();
-  Status stat = std::get<0>(returnstate);
+  Status stat = returnstate;
 
   return stat;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
index 6ea85fb..b27dd49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
@@ -93,6 +93,17 @@ public:
   virtual Status GetBlockLocations(const std::string & path,
     std::shared_ptr<FileBlockLocation> * locations) override;
 
+  virtual void Mkdirs(const std::string & path, long permissions, bool createparent,
+      std::function<void(const Status &)> handler) override;
+  virtual Status Mkdirs(const std::string & path, long permissions, bool createparent) override;
+
+  virtual void Delete(const std::string &path, bool recursive,
+      const std::function<void(const Status &)> &handler) override;
+  virtual Status Delete(const std::string &path, bool recursive) override;
+
+  virtual void Rename(const std::string &oldPath, const std::string &newPath,
+      const std::function<void(const Status &)> &handler) override;
+  virtual Status Rename(const std::string &oldPath, const std::string &newPath) override;
 
   /*****************************************************************************
    *                    FILE SYSTEM SNAPSHOT FUNCTIONS

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
index 98a96e1..4947f11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
@@ -64,6 +64,11 @@ void NameNodeOperations::GetBlockLocations(const std::string & path,
   LOG_TRACE(kFileSystem, << "NameNodeOperations::GetBlockLocations("
                            << FMT_THIS_ADDR << ", path=" << path << ", ...) called");
 
+  if (path.empty()) {
+    handler(Status::InvalidArgument("GetBlockLocations: argument 'path' cannot be empty"), nullptr);
+    return;
+  }
+
   GetBlockLocationsRequestProto req;
   req.set_src(path);
   req.set_offset(0);
@@ -106,6 +111,11 @@ void NameNodeOperations::GetFileInfo(const std::string & path,
   LOG_TRACE(kFileSystem, << "NameNodeOperations::GetFileInfo("
                            << FMT_THIS_ADDR << ", path=" << path << ") called");
 
+  if (path.empty()) {
+    handler(Status::InvalidArgument("GetFileInfo: argument 'path' cannot be empty"), StatInfo());
+    return;
+  }
+
   GetFileInfoRequestProto req;
   req.set_src(path);
 
@@ -164,6 +174,12 @@ void NameNodeOperations::GetListing(
       kFileSystem,
       << "NameNodeOperations::GetListing(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
+  if (path.empty()) {
+    std::shared_ptr<std::vector<StatInfo>> stat_infos;
+    handler(Status::InvalidArgument("GetListing: argument 'path' cannot be empty"), stat_infos, false);
+    return;
+  }
+
   GetListingRequestProto req;
   req.set_src(path);
   req.set_startafter(start_after.c_str());
@@ -198,6 +214,125 @@ void NameNodeOperations::GetListing(
       });
 }
 
+void NameNodeOperations::Mkdirs(const std::string & path, long permissions, bool createparent,
+  std::function<void(const Status &)> handler)
+{
+  using ::hadoop::hdfs::MkdirsRequestProto;
+  using ::hadoop::hdfs::MkdirsResponseProto;
+
+  LOG_TRACE(kFileSystem,
+      << "NameNodeOperations::Mkdirs(" << FMT_THIS_ADDR << ", path=" << path <<
+      ", permissions=" << permissions << ", createparent=" << createparent << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("Mkdirs: argument 'path' cannot be empty"));
+    return;
+  }
+
+  MkdirsRequestProto req;
+  req.set_src(path);
+  hadoop::hdfs::FsPermissionProto *perm = req.mutable_masked();
+  if (permissions < 0) {
+    perm->set_perm(0755);
+  } else {
+    perm->set_perm(permissions);
+  }
+  req.set_createparent(createparent);
+
+  auto resp = std::make_shared<MkdirsResponseProto>();
+
+  namenode_.Mkdirs(&req, resp, [resp, handler, path](const Status &stat) {
+    if (stat.ok()) {
+      // Checking resp
+      if(resp -> has_result() && resp ->result() == 1) {
+        handler(stat);
+      } else {
+        //NameNode does not specify why there is no result, in my testing it was happening when the path is not found
+        std::string errormsg = "No such file or directory: " + path;
+        Status statNew = Status::PathNotFound(errormsg.c_str());
+        handler(statNew);
+      }
+    } else {
+      handler(stat);
+    }
+  });
+}
+
+void NameNodeOperations::Delete(const std::string & path, bool recursive, std::function<void(const Status &)> handler) {
+  using ::hadoop::hdfs::DeleteRequestProto;
+  using ::hadoop::hdfs::DeleteResponseProto;
+
+  LOG_TRACE(kFileSystem,
+      << "NameNodeOperations::Delete(" << FMT_THIS_ADDR << ", path=" << path << ", recursive=" << recursive << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("Delete: argument 'path' cannot be empty"));
+    return;
+  }
+
+  DeleteRequestProto req;
+  req.set_src(path);
+  req.set_recursive(recursive);
+
+  auto resp = std::make_shared<DeleteResponseProto>();
+
+  namenode_.Delete(&req, resp, [resp, handler, path](const Status &stat) {
+    if (stat.ok()) {
+      // Checking resp
+      if(resp -> has_result() && resp ->result() == 1) {
+        handler(stat);
+      } else {
+        //NameNode does not specify why there is no result, in my testing it was happening when the path is not found
+        std::string errormsg = "No such file or directory: " + path;
+        Status statNew = Status::PathNotFound(errormsg.c_str());
+        handler(statNew);
+      }
+    } else {
+      handler(stat);
+    }
+  });
+}
+
+void NameNodeOperations::Rename(const std::string & oldPath, const std::string & newPath, std::function<void(const Status &)> handler) {
+  using ::hadoop::hdfs::RenameRequestProto;
+  using ::hadoop::hdfs::RenameResponseProto;
+
+  LOG_TRACE(kFileSystem,
+      << "NameNodeOperations::Rename(" << FMT_THIS_ADDR << ", oldPath=" << oldPath << ", newPath=" << newPath << ") called");
+
+  if (oldPath.empty()) {
+    handler(Status::InvalidArgument("Rename: argument 'oldPath' cannot be empty"));
+    return;
+  }
+
+  if (newPath.empty()) {
+    handler(Status::InvalidArgument("Rename: argument 'newPath' cannot be empty"));
+    return;
+  }
+
+  RenameRequestProto req;
+  req.set_src(oldPath);
+  req.set_dst(newPath);
+
+  auto resp = std::make_shared<RenameResponseProto>();
+
+  namenode_.Rename(&req, resp, [resp, handler](const Status &stat) {
+    if (stat.ok()) {
+      // Checking resp
+      if(resp -> has_result() && resp ->result() == 1) {
+        handler(stat);
+      } else {
+        //Since NameNode does not specify why the result is not success, we set the general error
+        std::string errormsg = "oldPath and parent directory of newPath must exist. newPath must not exist.";
+        Status statNew = Status::InvalidArgument(errormsg.c_str());
+        handler(statNew);
+      }
+    } else {
+      handler(stat);
+    }
+  });
+}
+
 void NameNodeOperations::CreateSnapshot(const std::string & path,
     const std::string & name, std::function<void(const Status &)> handler) {
   using ::hadoop::hdfs::CreateSnapshotRequestProto;
@@ -207,7 +342,7 @@ void NameNodeOperations::CreateSnapshot(const std::string & path,
       << "NameNodeOperations::CreateSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("CreateSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
@@ -220,7 +355,7 @@ void NameNodeOperations::CreateSnapshot(const std::string & path,
   auto resp = std::make_shared<CreateSnapshotResponseProto>();
 
   namenode_.CreateSnapshot(&req, resp,
-      [resp, handler, path](const Status &stat) {
+      [handler](const Status &stat) {
         handler(stat);
       });
 }
@@ -234,11 +369,11 @@ void NameNodeOperations::DeleteSnapshot(const std::string & path,
       << "NameNodeOperations::DeleteSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ", name=" << name << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("DeleteSnapshot: argument 'path' cannot be empty"));
     return;
   }
   if (name.empty()) {
-    handler(Status::InvalidArgument("Argument 'name' cannot be empty"));
+    handler(Status::InvalidArgument("DeleteSnapshot: argument 'name' cannot be empty"));
     return;
   }
 
@@ -249,7 +384,7 @@ void NameNodeOperations::DeleteSnapshot(const std::string & path,
   auto resp = std::make_shared<DeleteSnapshotResponseProto>();
 
   namenode_.DeleteSnapshot(&req, resp,
-      [resp, handler, path](const Status &stat) {
+      [handler](const Status &stat) {
         handler(stat);
       });
 }
@@ -262,7 +397,7 @@ void NameNodeOperations::AllowSnapshot(const std::string & path, std::function<v
       << "NameNodeOperations::AllowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("AllowSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
@@ -272,7 +407,7 @@ void NameNodeOperations::AllowSnapshot(const std::string & path, std::function<v
   auto resp = std::make_shared<AllowSnapshotResponseProto>();
 
   namenode_.AllowSnapshot(&req, resp,
-      [resp, handler, path](const Status &stat) {
+      [handler](const Status &stat) {
         handler(stat);
       });
 }
@@ -285,7 +420,7 @@ void NameNodeOperations::DisallowSnapshot(const std::string & path, std::functio
       << "NameNodeOperations::DisallowSnapshot(" << FMT_THIS_ADDR << ", path=" << path << ") called");
 
   if (path.empty()) {
-    handler(Status::InvalidArgument("Argument 'path' cannot be empty"));
+    handler(Status::InvalidArgument("DisallowSnapshot: argument 'path' cannot be empty"));
     return;
   }
 
@@ -295,7 +430,7 @@ void NameNodeOperations::DisallowSnapshot(const std::string & path, std::functio
   auto resp = std::make_shared<DisallowSnapshotResponseProto>();
 
   namenode_.DisallowSnapshot(&req, resp,
-      [resp, handler, path](const Status &stat) {
+      [handler](const Status &stat) {
         handler(stat);
       });
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
index d1366fc..e9b7f7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
@@ -65,6 +65,15 @@ public:
         std::function<void(const Status &, std::shared_ptr<std::vector<StatInfo>>&, bool)> handler,
         const std::string & start_after = "");
 
+  void Mkdirs(const std::string & path, long permissions, bool createparent,
+    std::function<void(const Status &)> handler);
+
+  void Delete(const std::string & path, bool recursive,
+      std::function<void(const Status &)> handler);
+
+  void Rename(const std::string & oldPath, const std::string & newPath,
+      std::function<void(const Status &)> handler);
+
   void CreateSnapshot(const std::string & path, const std::string & name,
       std::function<void(const Status &)> handler);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
index 1f65d4b..56132cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
@@ -172,6 +172,83 @@ TEST_F(HdfsExtTest, TestSnapshotOperations) {
   hdfsFreeFileInfo(file_infos, 0);
 }
 
+//Testing creating directories
+TEST_F(HdfsExtTest, TestMkdirs) {
+  HdfsHandle connection = cluster.connect_c();
+  hdfsFS fs = connection.handle();
+  EXPECT_NE(nullptr, fs);
+
+  //Correct operation
+  EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir123"));
+
+  //TODO Should return error if directory already exists?
+  //EXPECT_EQ(-1, hdfsCreateDirectory(fs, "/myDir123"));
+  //EXPECT_EQ((int) std::errc::file_exists, errno);
+
+  //Creating directory on a path of the existing file
+  std::string path = connection.newFile(1024); //1024 byte file
+  EXPECT_EQ(-1, hdfsCreateDirectory(fs, path.c_str()));
+  EXPECT_EQ((int) std::errc::file_exists, errno);
+}
+
+//Testing deleting files and directories
+TEST_F(HdfsExtTest, TestDelete) {
+  HdfsHandle connection = cluster.connect_c();
+  hdfsFS fs = connection.handle();
+  EXPECT_NE(nullptr, fs);
+
+  //Path not found
+  EXPECT_EQ(-1, hdfsDelete(fs, "/wrong_path", 1));
+  EXPECT_EQ((int) std::errc::no_such_file_or_directory, errno);
+
+  EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir"));
+  std::string path = connection.newFile("/myDir", 1024); //1024 byte file
+
+  //Non-recursive delete should fail on a non-empty directory
+  //error ENOTEMPTY(39) for libhdfspp or 255 for libhdfs
+  EXPECT_EQ(-1, hdfsDelete(fs, "/myDir", 0));
+  EXPECT_EQ((int) std::errc::directory_not_empty, errno);
+
+  //Correct operation
+  EXPECT_EQ(0, hdfsDelete(fs, "/myDir", 1));
+}
+
+//Testing renaming files and directories
+TEST_F(HdfsExtTest, TestRename) {
+  HdfsHandle connection = cluster.connect_c();
+  hdfsFS fs = connection.handle();
+  EXPECT_NE(nullptr, fs);
+
+  //Creating directory with two files
+  EXPECT_EQ(0, hdfsCreateDirectory(fs, "/myDir"));
+  std::string file1 = connection.newFile("/myDir", 1024); //1024 byte file
+  std::string file2 = connection.newFile("/myDir", 1024); //1024 byte file
+  std::string file3 = connection.newFile(1024); //1024 byte file
+
+  //Path not found
+  EXPECT_EQ(-1, hdfsRename(fs, "/wrong_path", "/new_name"));
+  EXPECT_EQ((int) std::errc::invalid_argument, errno);
+
+  //No parent directory in new path
+  EXPECT_EQ(-1, hdfsRename(fs, file1.c_str(), "/wrong_parent/new_name"));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+
+  //New name already exists in the folder
+  EXPECT_EQ(-1, hdfsRename(fs, file1.c_str(), file2.c_str()));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+
+  //Correct operation
+  EXPECT_EQ(0, hdfsRename(fs, file1.c_str(), "/myDir/new_awesome_name"));
+  EXPECT_EQ(0, hdfsRename(fs, file3.c_str(), "/myDir/another_file"));
+  EXPECT_EQ(0, hdfsRename(fs, "/myDir", "/new_awesome_dir"));
+
+  //Verification
+  int numEntries;
+  hdfsFileInfo * dirList = hdfsListDirectory(fs, "/new_awesome_dir", &numEntries);
+  EXPECT_NE(nullptr, dirList);
+  EXPECT_EQ(3, numEntries);
+  hdfsFreeFileInfo(dirList, 3);
+}
 
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1c87ccb/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
index 12e9f71..a518d92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
@@ -268,11 +268,11 @@ int hdfsMove(hdfsFS srcFS, const char* src, hdfsFS dstFS, const char* dst) {
 }
 
 int hdfsDelete(hdfsFS fs, const char* path, int recursive) {
-  return libhdfs_hdfsDelete(fs->libhdfsRep, path, recursive);
+  return libhdfspp_hdfsDelete(fs->libhdfsppRep, path, recursive);
 }
 
 int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath) {
-  return libhdfs_hdfsRename(fs->libhdfsRep, oldPath, newPath);
+  return libhdfspp_hdfsRename(fs->libhdfsppRep, oldPath, newPath);
 }
 
 char* hdfsGetWorkingDirectory(hdfsFS fs, char *buffer, size_t bufferSize) {
@@ -284,7 +284,7 @@ int hdfsSetWorkingDirectory(hdfsFS fs, const char* path) {
 }
 
 int hdfsCreateDirectory(hdfsFS fs, const char* path) {
-  return libhdfs_hdfsCreateDirectory(fs->libhdfsRep, path);
+  return libhdfspp_hdfsCreateDirectory(fs->libhdfsppRep, path);
 }
 
 int hdfsSetReplication(hdfsFS fs, const char* path, int16_t replication) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[3/3] hadoop git commit: HDFS-10524: libhdfs++: Implement chmod and chown. Contributed by Anatoli Shein.

Posted by bo...@apache.org.
HDFS-10524: libhdfs++: Implement chmod and chown.  Contributed by Anatoli Shein.


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

Branch: refs/heads/HDFS-8707
Commit: adb1a63e1c2a8581b90418489adf4e3ff70b4547
Parents: a1c87cc
Author: Bob Hansen <bo...@hpe.com>
Authored: Tue Jun 21 16:42:47 2016 -0400
Committer: Bob Hansen <bo...@hpe.com>
Committed: Tue Jun 21 16:42:47 2016 -0400

----------------------------------------------------------------------
 .../libhdfs-tests/test_libhdfs_threaded.c       |  23 +--
 .../native/libhdfspp/include/hdfspp/hdfspp.h    |  24 +++
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     |  65 ++++++++
 .../main/native/libhdfspp/lib/fs/filesystem.cc  |  73 +++++++++
 .../main/native/libhdfspp/lib/fs/filesystem.h   |   9 ++
 .../libhdfspp/lib/fs/namenode_operations.cc     |  75 ++++++++++
 .../libhdfspp/lib/fs/namenode_operations.h      |   8 +
 .../native/libhdfspp/tests/hdfs_ext_test.cc     |  45 ++++++
 .../src/main/native/libhdfspp/tests/hdfs_shim.c | 148 +++++++++++++------
 9 files changed, 413 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
index c3c2db8..759fd31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_threaded.c
@@ -291,21 +291,14 @@ static int doTestHdfsOperations(struct tlhThreadInfo *ti, hdfsFS fs,
     //Test case: File does not exist
     EXPECT_NULL_WITH_ERRNO(hdfsGetPathInfo(fs, invalid_path), ENOENT);
 
-//  Test case: No permission to access parent directory
-//  Trying to set permissions of the parent directory to 0
-//  by a super user, and then connecting as SomeGuy. Should
-//  receive permission denied, but receives fileInfo.
-//  EXPECT_ZERO(hdfsChmod(fs, paths->prefix, 0));
-//  EXPECT_ZERO(hdfsChmod(fs, paths->file2, 0));
-//  EXPECT_ZERO(hdfsDisconnect(fs));
-//  EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs, "SomeGuy"));
-//  EXPECT_NULL_WITH_ERRNO(hdfsGetPathInfo(fs, paths->file2), EACCES);
-//  EXPECT_ZERO(hdfsDisconnect(fs));
-//  EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs, NULL));
-//  if (!fs) {
-//      return 1;
-//  }
-  return 0;
+    //Test case: No permission to access parent directory
+    EXPECT_ZERO(hdfsChmod(fs, paths->prefix, 0));
+    //reconnect as user "SomeGuy" and verify that we get permission errors
+    hdfsFS fs2 = NULL;
+    EXPECT_ZERO(hdfsSingleNameNodeConnect(tlhCluster, &fs2, "SomeGuy"));
+    EXPECT_NULL_WITH_ERRNO(hdfsGetPathInfo(fs2, paths->file2), EACCES);
+    EXPECT_ZERO(hdfsDisconnect(fs2));
+    return 0;
 }
 
 static int testHdfsOperationsImpl(struct tlhThreadInfo *ti)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
index 73a9de9..a44df74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
@@ -250,6 +250,30 @@ class FileSystem {
       const std::function<void(const Status &)> &handler) = 0;
   virtual Status Rename(const std::string &oldPath, const std::string &newPath) = 0;
 
+  /**
+   * Set permissions for an existing file/directory.
+   *
+   * @param path          the path to the file or directory
+   * @param permissions   the bitmask to set it to (should be between 0 and 01777)
+   */
+  virtual void SetPermission(const std::string & path,
+      short permissions, const std::function<void(const Status &)> &handler) = 0;
+  virtual Status SetPermission(const std::string & path, short permissions) = 0;
+
+  /**
+   * Set Owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname can be empty.
+   * @param path      file path
+   * @param username  If it is empty, the original username remains unchanged.
+   * @param groupname If it is empty, the original groupname remains unchanged.
+   */
+  virtual void SetOwner(const std::string & path, const std::string & username,
+      const std::string & groupname, const std::function<void(const Status &)> &handler) = 0;
+  virtual Status SetOwner(const std::string & path,
+      const std::string & username, const std::string & groupname) = 0;
+
+
+
   /*****************************************************************************
    *                    FILE SYSTEM SNAPSHOT FUNCTIONS
    ****************************************************************************/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
index deb7eb0..04065b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
@@ -284,6 +284,16 @@ hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user) {
   return doHdfsConnect(std::string(nn), port, std::string(user), Options());
 }
 
+hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user ) {
+  //libhdfspp always returns a new instance
+  return doHdfsConnect(std::string(nn), port, std::string(user), Options());
+}
+
+hdfsFS hdfsConnectNewInstance(const char* nn, tPort port) {
+  //libhdfspp always returns a new instance
+  return hdfsConnectAsUser(nn, port, "");
+}
+
 int hdfsDisconnect(hdfsFS fs) {
   try
   {
@@ -587,6 +597,56 @@ int hdfsRename(hdfsFS fs, const char* oldPath, const char* newPath) {
   }
 }
 
+int hdfsChmod(hdfsFS fs, const char* path, short mode){
+  try {
+      errno = 0;
+      if (!CheckSystem(fs)) {
+        return -1;
+      }
+      if (!path) {
+        return Error(Status::InvalidArgument("hdfsChmod: argument 'path' cannot be NULL"));
+      }
+      Status stat = NameNodeOperations::CheckValidPermissionMask(mode);
+      if (!stat.ok()) {
+        return Error(stat);
+      }
+      stat = fs->get_impl()->SetPermission(path, mode);
+      if (!stat.ok()) {
+        return Error(stat);
+      }
+      return 0;
+    } catch (const std::exception & e) {
+      return ReportException(e);
+    } catch (...) {
+      return ReportCaughtNonException();
+    }
+}
+
+int hdfsChown(hdfsFS fs, const char* path, const char *owner, const char *group){
+  try {
+      errno = 0;
+      if (!CheckSystem(fs)) {
+        return -1;
+      }
+      if (!path) {
+        return Error(Status::InvalidArgument("hdfsChown: argument 'path' cannot be NULL"));
+      }
+      std::string own = (owner) ? owner : "";
+      std::string grp = (group) ? group : "";
+
+      Status stat;
+      stat = fs->get_impl()->SetOwner(path, own, grp);
+      if (!stat.ok()) {
+        return Error(stat);
+      }
+      return 0;
+    } catch (const std::exception & e) {
+      return ReportException(e);
+    } catch (...) {
+      return ReportCaughtNonException();
+    }
+}
+
 int hdfsCreateSnapshot(hdfsFS fs, const char* path, const char* name) {
   try {
     errno = 0;
@@ -1009,6 +1069,11 @@ void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName)
   }
 }
 
+void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld) {
+  //libhdfspp always returns a new instance, so nothing to do
+  (void)bld;
+  errno = 0;
+}
 
 void hdfsFreeBuilder(struct hdfsBuilder *bld)
 {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
index 1d11c21..f3c32c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
@@ -617,6 +617,79 @@ Status FileSystemImpl::Rename(const std::string &oldPath, const std::string &new
   return stat;
 }
 
+void FileSystemImpl::SetPermission(const std::string & path,
+    short permissions, const std::function<void(const Status &)> &handler) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::SetPermission(" << FMT_THIS_ADDR << ", path=" << path << ", permissions=" << permissions << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("SetPermission: argument 'path' cannot be empty"));
+    return;
+  }
+  Status permStatus = NameNodeOperations::CheckValidPermissionMask(permissions);
+  if (!permStatus.ok()) {
+    handler(permStatus);
+    return;
+  }
+
+  nn_.SetPermission(path, permissions, handler);
+}
+
+Status FileSystemImpl::SetPermission(const std::string & path, short permissions) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::[sync]SetPermission(" << FMT_THIS_ADDR << ", path=" << path << ", permissions=" << permissions << ") called");
+
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
+
+  /* wrap async FileSystem::SetPermission with promise to make it a blocking call */
+  auto h = [callstate](const Status &s) {
+    callstate->set_value(s);
+  };
+
+  SetPermission(path, permissions, h);
+
+  /* block until promise is set */
+  Status stat = future.get();
+
+  return stat;
+}
+
+void FileSystemImpl::SetOwner(const std::string & path, const std::string & username,
+    const std::string & groupname, const std::function<void(const Status &)> &handler) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::SetOwner(" << FMT_THIS_ADDR << ", path=" << path << ", username=" << username << ", groupname=" << groupname << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("SetOwner: argument 'path' cannot be empty"));
+    return;
+  }
+
+  nn_.SetOwner(path, username, groupname, handler);
+}
+
+Status FileSystemImpl::SetOwner(const std::string & path,
+    const std::string & username, const std::string & groupname) {
+  LOG_DEBUG(kFileSystem,
+      << "FileSystemImpl::[sync]SetOwner(" << FMT_THIS_ADDR << ", path=" << path << ", username=" << username << ", groupname=" << groupname << ") called");
+
+  auto callstate = std::make_shared<std::promise<Status>>();
+  std::future<Status> future(callstate->get_future());
+
+  /* wrap async FileSystem::SetOwner with promise to make it a blocking call */
+  auto h = [callstate](const Status &s) {
+    callstate->set_value(s);
+  };
+
+  SetOwner(path, username, groupname, h);
+
+  /* block until promise is set */
+  Status stat = future.get();
+
+  return stat;
+}
+
+
 void FileSystemImpl::CreateSnapshot(const std::string &path,
     const std::string &name,
     const std::function<void(const Status &)> &handler) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
index b27dd49..73ad7eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
@@ -105,6 +105,15 @@ public:
       const std::function<void(const Status &)> &handler) override;
   virtual Status Rename(const std::string &oldPath, const std::string &newPath) override;
 
+  virtual void SetPermission(const std::string & path,
+      short permissions, const std::function<void(const Status &)> &handler) override;
+  virtual Status SetPermission(const std::string & path, short permissions) override;
+
+  virtual void SetOwner(const std::string & path, const std::string & username,
+      const std::string & groupname, const std::function<void(const Status &)> &handler) override;
+  virtual Status SetOwner(const std::string & path,
+      const std::string & username, const std::string & groupname) override;
+
   /*****************************************************************************
    *                    FILE SYSTEM SNAPSHOT FUNCTIONS
    ****************************************************************************/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
index 4947f11..ee566ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
@@ -38,6 +38,18 @@ namespace hdfs {
  *                    NAMENODE OPERATIONS
  ****************************************************************************/
 
+Status NameNodeOperations::CheckValidPermissionMask(short permissions) {
+  if (permissions < 0 || permissions > 01777) {
+    std::stringstream errormsg;
+    errormsg << "IsValidPermissionMask: argument 'permissions' is " << std::oct
+        << std::showbase << permissions << " (should be between 0 and 01777)";
+    //Avoid copying by binding errormsg.str() to a const reference, which extends its lifetime
+    const std::string& tmp = errormsg.str();
+    return Status::InvalidArgument(tmp.c_str());
+  }
+  return Status::OK();
+}
+
 void NameNodeOperations::Connect(const std::string &cluster_name,
                                  const std::string &server,
                              const std::string &service,
@@ -333,6 +345,69 @@ void NameNodeOperations::Rename(const std::string & oldPath, const std::string &
   });
 }
 
+void NameNodeOperations::SetPermission(const std::string & path,
+    short permissions, std::function<void(const Status &)> handler) {
+  using ::hadoop::hdfs::SetPermissionRequestProto;
+  using ::hadoop::hdfs::SetPermissionResponseProto;
+
+  LOG_TRACE(kFileSystem,
+      << "NameNodeOperations::SetPermission(" << FMT_THIS_ADDR << ", path=" << path << ", permissions=" << permissions << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("SetPermission: argument 'path' cannot be empty"));
+    return;
+  }
+  Status permStatus = CheckValidPermissionMask(permissions);
+  if (!permStatus.ok()) {
+    handler(permStatus);
+    return;
+  }
+
+  SetPermissionRequestProto req;
+  req.set_src(path);
+
+  hadoop::hdfs::FsPermissionProto *perm = req.mutable_permission();
+  perm->set_perm(permissions);
+
+  auto resp = std::make_shared<SetPermissionResponseProto>();
+
+  namenode_.SetPermission(&req, resp,
+      [handler](const Status &stat) {
+        handler(stat);
+      });
+}
+
+void NameNodeOperations::SetOwner(const std::string & path,
+    const std::string & username, const std::string & groupname, std::function<void(const Status &)> handler) {
+  using ::hadoop::hdfs::SetOwnerRequestProto;
+  using ::hadoop::hdfs::SetOwnerResponseProto;
+
+  LOG_TRACE(kFileSystem,
+      << "NameNodeOperations::SetOwner(" << FMT_THIS_ADDR << ", path=" << path << ", username=" << username << ", groupname=" << groupname << ") called");
+
+  if (path.empty()) {
+    handler(Status::InvalidArgument("SetOwner: argument 'path' cannot be empty"));
+    return;
+  }
+
+  SetOwnerRequestProto req;
+  req.set_src(path);
+
+  if(!username.empty()) {
+    req.set_username(username);
+  }
+  if(!groupname.empty()) {
+    req.set_groupname(groupname);
+  }
+
+  auto resp = std::make_shared<SetOwnerResponseProto>();
+
+  namenode_.SetOwner(&req, resp,
+      [handler](const Status &stat) {
+        handler(stat);
+      });
+}
+
 void NameNodeOperations::CreateSnapshot(const std::string & path,
     const std::string & name, std::function<void(const Status &)> handler) {
   using ::hadoop::hdfs::CreateSnapshotRequestProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
index e9b7f7c..4f4d63e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
@@ -47,6 +47,8 @@ public:
   engine_(io_service, options, client_name, user_name, protocol_name, protocol_version),
   namenode_(& engine_) {}
 
+  static Status CheckValidPermissionMask(short permissions);
+
   void Connect(const std::string &cluster_name,
                const std::string &server,
                const std::string &service,
@@ -74,6 +76,12 @@ public:
   void Rename(const std::string & oldPath, const std::string & newPath,
       std::function<void(const Status &)> handler);
 
+  void SetPermission(const std::string & path, short permissions,
+      std::function<void(const Status &)> handler);
+
+  void SetOwner(const std::string & path, const std::string & username,
+      const std::string & groupname, std::function<void(const Status &)> handler);
+
   void CreateSnapshot(const std::string & path, const std::string & name,
       std::function<void(const Status &)> handler);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
index 56132cf..f154be8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
@@ -250,9 +250,54 @@ TEST_F(HdfsExtTest, TestRename) {
   hdfsFreeFileInfo(dirList, 3);
 }
 
+//Testing Chmod and Chown
+TEST_F(HdfsExtTest, TestChmodChown) {
+  HdfsHandle connection = cluster.connect_c();
+  hdfsFS fs = connection.handle();
+  EXPECT_NE(nullptr, fs);
+
+  //Path not found
+  std::string path = "/wrong/dir/";
+  EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), 0777));
+  EXPECT_EQ((int ) std::errc::no_such_file_or_directory, errno);
+  EXPECT_EQ(-1, hdfsChown(fs, path.c_str(), "foo", "bar"));
+  EXPECT_EQ((int ) std::errc::no_such_file_or_directory, errno);
+
+  //Wrong arguments
+  path = connection.newFile(1024); //1024 byte file
+  EXPECT_EQ(-1, hdfsChmod(fs, nullptr, 0777));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+  EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), 07777));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+  EXPECT_EQ(-1, hdfsChmod(fs, path.c_str(), -1));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+  EXPECT_EQ(-1, hdfsChown(fs, nullptr, "foo", "bar"));
+  EXPECT_EQ((int ) std::errc::invalid_argument, errno);
+
+  //Permission denied
+  HdfsHandle connection2 = cluster.connect_c("OtherGuy");
+  hdfsFS fs2 = connection2.handle();
+  EXPECT_EQ(-1, hdfsChmod(fs2, path.c_str(), 0123));
+  EXPECT_EQ((int ) std::errc::permission_denied, errno);
+  EXPECT_EQ(-1, hdfsChown(fs2, path.c_str(), "cool", "nice"));
+  EXPECT_EQ((int ) std::errc::permission_denied, errno);
+
+  //Verify Chmod and Chown worked
+  EXPECT_EQ(0, hdfsChmod(fs, path.c_str(), 0123));
+  EXPECT_EQ(0, hdfsChown(fs, path.c_str(), "cool", "nice"));
+  hdfsFileInfo *file_info;
+  EXPECT_NE(nullptr, file_info = hdfsGetPathInfo(fs, path.c_str()));
+  EXPECT_EQ(0123, file_info->mPermissions);
+  EXPECT_STREQ("cool", file_info->mOwner);
+  EXPECT_STREQ("nice", file_info->mGroup);
+  hdfsFreeFileInfo(file_info, 1);
 }
 
 
+}
+
+
+
 int main(int argc, char *argv[]) {
   // The following line must be executed to initialize Google Mock
   // (and Google Test) before running the tests.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/adb1a63e/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
index a518d92..32f920f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
@@ -24,17 +24,6 @@
 #include <stdlib.h>
 #include <string.h>
 
-/* Cheat for now and use the same hdfsBuilder as libhdfs */
-/* (libhdfspp doesn't have an hdfsBuilder yet). */
-struct hdfsBuilder {
-    int forceNewInstance;
-    const char *nn;
-    tPort port;
-    const char *kerbTicketCachePath;
-    const char *userName;
-    struct hdfsBuilderConfOpt *opts;
-};
-
 /* Shim structs and functions that delegate to libhdfspp and libhdfs. */
 struct hdfs_internal {
   libhdfs_hdfsFS libhdfsRep;
@@ -48,6 +37,11 @@ struct hdfsFile_internal {
 };
 typedef struct hdfsFile_internal* hdfsFile;
 
+struct hdfsBuilder {
+  struct hdfsBuilder *  libhdfs_builder;
+  struct hdfsBuilder * libhdfspp_builder;
+};
+
 #define REPORT_FUNCTION_NOT_IMPLEMENTED                     \
   fprintf(stderr, "%s failed: function not implemented by " \
     "libhdfs++ test shim", __PRETTY_FUNCTION__);
@@ -78,74 +72,139 @@ void hdfsFileFreeReadStatistics(struct hdfsReadStatistics *stats) {
 }
 
 hdfsFS hdfsConnectAsUser(const char* nn, tPort port, const char *user) {
-  return (hdfsFS) libhdfspp_hdfsConnectAsUser(nn, port, user);
+  hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
+  ret->libhdfsRep = libhdfs_hdfsConnectAsUser(nn, port, user);
+    if (!ret->libhdfsRep) {
+      libhdfs_hdfsDisconnect(ret->libhdfsRep);
+      free(ret);
+      return NULL;
+    }
+  ret->libhdfsppRep = libhdfspp_hdfsConnectAsUser(nn, port, user);
+  if (!ret->libhdfsppRep) {
+    libhdfs_hdfsDisconnect(ret->libhdfsRep);
+    free(ret);
+    return NULL;
+  }
+  return ret;
 }
 
 hdfsFS hdfsConnect(const char* nn, tPort port) {
-  REPORT_FUNCTION_NOT_IMPLEMENTED
-  return NULL;
+  hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
+  ret->libhdfsRep = libhdfs_hdfsConnect(nn, port);
+    if (!ret->libhdfsRep) {
+      libhdfs_hdfsDisconnect(ret->libhdfsRep);
+      free(ret);
+      return NULL;
+    }
+  ret->libhdfsppRep = libhdfspp_hdfsConnect(nn, port);
+  if (!ret->libhdfsppRep) {
+    libhdfs_hdfsDisconnect(ret->libhdfsRep);
+    free(ret);
+    return NULL;
+  }
+  return ret;
 }
 
 hdfsFS hdfsConnectAsUserNewInstance(const char* nn, tPort port, const char *user ) {
-  REPORT_FUNCTION_NOT_IMPLEMENTED
-  return NULL;
+  hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
+  ret->libhdfsRep = libhdfs_hdfsConnectAsUserNewInstance(nn, port, user);
+    if (!ret->libhdfsRep) {
+      libhdfs_hdfsDisconnect(ret->libhdfsRep);
+      free(ret);
+      return NULL;
+    }
+  ret->libhdfsppRep = libhdfspp_hdfsConnectAsUserNewInstance(nn, port, user);
+  if (!ret->libhdfsppRep) {
+    libhdfs_hdfsDisconnect(ret->libhdfsRep);
+    free(ret);
+    return NULL;
+  }
+  return ret;
 }
 
 hdfsFS hdfsConnectNewInstance(const char* nn, tPort port) {
-  REPORT_FUNCTION_NOT_IMPLEMENTED
-  return NULL;
+  hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
+  ret->libhdfsRep = libhdfs_hdfsConnectNewInstance(nn, port);
+    if (!ret->libhdfsRep) {
+      libhdfs_hdfsDisconnect(ret->libhdfsRep);
+      free(ret);
+      return NULL;
+    }
+  ret->libhdfsppRep = libhdfspp_hdfsConnectNewInstance(nn, port);
+  if (!ret->libhdfsppRep) {
+    libhdfs_hdfsDisconnect(ret->libhdfsRep);
+    free(ret);
+    return NULL;
+  }
+  return ret;
 }
 
 hdfsFS hdfsBuilderConnect(struct hdfsBuilder *bld) {
   hdfsFS ret = calloc(1, sizeof(struct hdfs_internal));
-  ret->libhdfsppRep = libhdfspp_hdfsConnect(bld->nn, bld->port);
+  ret->libhdfsRep = libhdfs_hdfsBuilderConnect(bld->libhdfs_builder);
+  if (!ret->libhdfsRep) {
+    free(ret);
+    return NULL;
+  }
+  /* Destroys bld object. */
+  ret->libhdfsppRep = libhdfspp_hdfsBuilderConnect(bld->libhdfspp_builder);
   if (!ret->libhdfsppRep) {
+    libhdfs_hdfsDisconnect(ret->libhdfsRep);
     free(ret);
-    ret = NULL;
-  } else {
-    /* Destroys bld object. */
-    ret->libhdfsRep = libhdfs_hdfsBuilderConnect(bld);
-    if (!ret->libhdfsRep) {
-      libhdfspp_hdfsDisconnect(ret->libhdfsppRep);
-      free(ret);
-      ret = NULL;
-    }
+    return NULL;
   }
   return ret;
 }
 
 struct hdfsBuilder *hdfsNewBuilder(void) {
-  return libhdfs_hdfsNewBuilder();
+  struct hdfsBuilder * result = calloc(1, sizeof(struct hdfsBuilder));
+  result -> libhdfs_builder = libhdfs_hdfsNewBuilder();
+  result -> libhdfspp_builder = libhdfspp_hdfsNewBuilder();
+  return result;
 }
 
 void hdfsBuilderSetForceNewInstance(struct hdfsBuilder *bld) {
-  libhdfs_hdfsBuilderSetForceNewInstance(bld);
+  libhdfs_hdfsBuilderSetForceNewInstance(bld->libhdfs_builder);
+  libhdfspp_hdfsBuilderSetForceNewInstance(bld->libhdfspp_builder);
 }
 
 void hdfsBuilderSetNameNode(struct hdfsBuilder *bld, const char *nn) {
-  libhdfs_hdfsBuilderSetNameNode(bld, nn);
+  libhdfs_hdfsBuilderSetNameNode(bld->libhdfs_builder, nn);
+  libhdfspp_hdfsBuilderSetNameNode(bld->libhdfspp_builder, nn);
 }
 
 void hdfsBuilderSetNameNodePort(struct hdfsBuilder *bld, tPort port) {
-  libhdfs_hdfsBuilderSetNameNodePort(bld, port);
+  libhdfs_hdfsBuilderSetNameNodePort(bld->libhdfs_builder, port);
+  libhdfspp_hdfsBuilderSetNameNodePort(bld->libhdfspp_builder, port);
 }
 
 void hdfsBuilderSetUserName(struct hdfsBuilder *bld, const char *userName) {
-  libhdfs_hdfsBuilderSetUserName(bld, userName);
+  libhdfs_hdfsBuilderSetUserName(bld->libhdfs_builder, userName);
+  libhdfspp_hdfsBuilderSetUserName(bld->libhdfspp_builder, userName);
 }
 
 void hdfsBuilderSetKerbTicketCachePath(struct hdfsBuilder *bld,
                                const char *kerbTicketCachePath) {
-  libhdfs_hdfsBuilderSetKerbTicketCachePath(bld, kerbTicketCachePath);
+  REPORT_FUNCTION_NOT_IMPLEMENTED
 }
 
 void hdfsFreeBuilder(struct hdfsBuilder *bld) {
-  libhdfs_hdfsFreeBuilder(bld);
+  libhdfs_hdfsFreeBuilder(bld->libhdfs_builder);
+  libhdfspp_hdfsFreeBuilder(bld->libhdfspp_builder);
+  free(bld);
 }
 
 int hdfsBuilderConfSetStr(struct hdfsBuilder *bld, const char *key,
                           const char *val) {
-  return libhdfs_hdfsBuilderConfSetStr(bld, key, val);
+  int ret = libhdfs_hdfsBuilderConfSetStr(bld->libhdfs_builder, key, val);
+  if (ret) {
+    return ret;
+  }
+  ret = libhdfspp_hdfsBuilderConfSetStr(bld->libhdfspp_builder, key, val);
+  if (ret) {
+    return ret;
+  }
+  return 0;
 }
 
 int hdfsConfGetStr(const char *key, char **val) {
@@ -161,11 +220,16 @@ void hdfsConfStrFree(char *val) {
 }
 
 int hdfsDisconnect(hdfsFS fs) {
-  int ret;
-  libhdfspp_hdfsDisconnect(fs->libhdfsppRep);
-  ret = libhdfs_hdfsDisconnect(fs->libhdfsRep);
+  int ret1 = libhdfs_hdfsDisconnect(fs->libhdfsRep);
+  int ret2 = libhdfspp_hdfsDisconnect(fs->libhdfsppRep);
   free(fs);
-  return ret;
+  if (ret1){
+    return ret1;
+  } else if (ret2){
+    return ret2;
+  } else {
+    return 0;
+  }
 }
 
 hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags,
@@ -337,11 +401,11 @@ tOffset hdfsGetUsed(hdfsFS fs) {
 
 int hdfsChown(hdfsFS fs, const char* path, const char *owner,
               const char *group) {
-  return libhdfs_hdfsChown(fs->libhdfsRep, path, owner, group);
+  return libhdfspp_hdfsChown(fs->libhdfsppRep, path, owner, group);
 }
 
 int hdfsChmod(hdfsFS fs, const char* path, short mode) {
-  return libhdfs_hdfsChmod(fs->libhdfsRep, path, mode);
+  return libhdfspp_hdfsChmod(fs->libhdfsppRep, path, mode);
 }
 
 int hdfsUtime(hdfsFS fs, const char* path, tTime mtime, tTime atime) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org