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 jh...@apache.org on 2017/06/13 15:28:19 UTC

[4/4] hadoop git commit: HDFS-10785: libhdfs++: Implement the rest of the tools. Contributed by Anatoli Schein

HDFS-10785: libhdfs++: Implement the rest of the tools. Contributed by Anatoli Schein


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

Branch: refs/heads/HDFS-8707
Commit: 40e3290b925310bc0ef861c1db7b2b544d9566d4
Parents: 5ae34ac
Author: James Clampffer <ja...@hp.com>
Authored: Tue Jun 13 11:27:55 2017 -0400
Committer: James Clampffer <ja...@hp.com>
Committed: Tue Jun 13 11:27:55 2017 -0400

----------------------------------------------------------------------
 .../native/libhdfspp/examples/CMakeLists.txt    |   2 +-
 .../main/native/libhdfspp/examples/c/cat/cat.c  |   4 +-
 .../native/libhdfspp/examples/cc/CMakeLists.txt |  22 ++
 .../libhdfspp/examples/cc/cat/CMakeLists.txt    |  35 ++++
 .../native/libhdfspp/examples/cc/cat/cat.cc     | 109 ++++++++++
 .../examples/cc/connect_cancel/CMakeLists.txt   |  27 +++
 .../cc/connect_cancel/connect_cancel.cc         | 154 ++++++++++++++
 .../libhdfspp/examples/cc/find/CMakeLists.txt   |  35 ++++
 .../native/libhdfspp/examples/cc/find/find.cc   | 162 +++++++++++++++
 .../examples/cc/gendirs/CMakeLists.txt          |  35 ++++
 .../libhdfspp/examples/cc/gendirs/gendirs.cc    | 144 +++++++++++++
 .../libhdfspp/examples/cpp/CMakeLists.txt       |  22 --
 .../libhdfspp/examples/cpp/cat/CMakeLists.txt   |  35 ----
 .../native/libhdfspp/examples/cpp/cat/cat.cpp   | 109 ----------
 .../examples/cpp/connect_cancel/CMakeLists.txt  |  29 ---
 .../cpp/connect_cancel/connect_cancel.cc        | 158 --------------
 .../libhdfspp/examples/cpp/find/CMakeLists.txt  |  35 ----
 .../native/libhdfspp/examples/cpp/find/find.cpp | 162 ---------------
 .../examples/cpp/gendirs/CMakeLists.txt         |  35 ----
 .../libhdfspp/examples/cpp/gendirs/gendirs.cpp  | 144 -------------
 .../libhdfspp/include/hdfspp/content_summary.h  |  48 +++++
 .../native/libhdfspp/include/hdfspp/fsinfo.h    |  34 ++-
 .../native/libhdfspp/include/hdfspp/hdfs_ext.h  |  11 +
 .../native/libhdfspp/include/hdfspp/hdfspp.h    |  25 ++-
 .../native/libhdfspp/include/hdfspp/statinfo.h  |  46 ++---
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     |  31 +++
 .../native/libhdfspp/lib/common/CMakeLists.txt  |   2 +-
 .../libhdfspp/lib/common/content_summary.cc     |  55 +++++
 .../main/native/libhdfspp/lib/common/fsinfo.cc  |  61 ++++++
 .../native/libhdfspp/lib/common/statinfo.cc     |  74 +++++++
 .../main/native/libhdfspp/lib/fs/filesystem.cc  |  46 ++++-
 .../main/native/libhdfspp/lib/fs/filesystem.h   |  18 ++
 .../native/libhdfspp/lib/fs/filesystem_sync.cc  |  52 +++++
 .../libhdfspp/lib/fs/namenode_operations.cc     |  90 +++++++-
 .../libhdfspp/lib/fs/namenode_operations.h      |  10 +-
 .../native/libhdfspp/tests/hdfs_ext_test.cc     |  11 +-
 .../src/main/native/libhdfspp/tests/hdfs_shim.c |   4 +
 .../libhdfspp/tests/libhdfs_wrapper_undefs.h    |   1 +
 .../libhdfspp/tests/libhdfspp_wrapper_defines.h |   1 +
 .../main/native/libhdfspp/tools/CMakeLists.txt  |  65 +++++-
 .../libhdfspp/tools/hdfs_allowSnapshot.cc       |  90 ++++++++
 .../src/main/native/libhdfspp/tools/hdfs_cat.cc |  87 ++++++++
 .../main/native/libhdfspp/tools/hdfs_cat.cpp    | 120 -----------
 .../main/native/libhdfspp/tools/hdfs_chgrp.cc   | 189 +++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_chgrp.cpp  | 196 ------------------
 .../main/native/libhdfspp/tools/hdfs_chmod.cc   | 187 +++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_chmod.cpp  | 194 -----------------
 .../main/native/libhdfspp/tools/hdfs_chown.cc   | 199 ++++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_chown.cpp  | 206 -------------------
 .../native/libhdfspp/tools/hdfs_copyToLocal.cc  |  92 +++++++++
 .../main/native/libhdfspp/tools/hdfs_count.cc   |  97 +++++++++
 .../libhdfspp/tools/hdfs_createSnapshot.cc      |  99 +++++++++
 .../libhdfspp/tools/hdfs_deleteSnapshot.cc      |  91 ++++++++
 .../src/main/native/libhdfspp/tools/hdfs_df.cc  |  93 +++++++++
 .../libhdfspp/tools/hdfs_disallowSnapshot.cc    |  90 ++++++++
 .../src/main/native/libhdfspp/tools/hdfs_du.cc  | 180 ++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_find.cc    | 150 ++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_find.cpp   | 156 --------------
 .../src/main/native/libhdfspp/tools/hdfs_get.cc |  92 +++++++++
 .../src/main/native/libhdfspp/tools/hdfs_ls.cc  | 134 ++++++++++++
 .../main/native/libhdfspp/tools/hdfs_mkdir.cc   | 102 +++++++++
 .../native/libhdfspp/tools/hdfs_moveToLocal.cc  |  94 +++++++++
 .../libhdfspp/tools/hdfs_renameSnapshot.cc      |  92 +++++++++
 .../src/main/native/libhdfspp/tools/hdfs_rm.cc  |  94 +++++++++
 .../main/native/libhdfspp/tools/hdfs_setrep.cc  | 176 ++++++++++++++++
 .../main/native/libhdfspp/tools/hdfs_stat.cc    |  91 ++++++++
 .../main/native/libhdfspp/tools/hdfs_tail.cc    | 128 ++++++++++++
 .../main/native/libhdfspp/tools/tools_common.cc | 115 +++++++++++
 .../native/libhdfspp/tools/tools_common.cpp     |  70 -------
 .../main/native/libhdfspp/tools/tools_common.h  |   8 +-
 70 files changed, 4116 insertions(+), 1739 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/CMakeLists.txt
index 6b680cc..ca3a2c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/CMakeLists.txt
@@ -17,4 +17,4 @@
 #
 
 add_subdirectory(c)
-add_subdirectory(cpp)
+add_subdirectory(cc)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/c/cat/cat.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/c/cat/cat.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/c/cat/cat.c
index 586e0a9..bee5382 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/c/cat/cat.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/c/cat/cat.c
@@ -30,6 +30,8 @@
 #include "common/util_c.h"
 
 #define SCHEME "hdfs"
+#define BUF_SIZE 1048576 //1 MB
+static char input_buffer[BUF_SIZE];
 
 int main(int argc, char** argv) {
 
@@ -84,8 +86,6 @@ int main(int argc, char** argv) {
     return 1;
   }
 
-  char input_buffer[4096];
-
   ssize_t read_bytes_count = 0;
   ssize_t last_read_bytes = 0;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/CMakeLists.txt
new file mode 100644
index 0000000..8893f03
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/CMakeLists.txt
@@ -0,0 +1,22 @@
+#
+# 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.
+#
+
+add_subdirectory(cat)
+add_subdirectory(gendirs)
+add_subdirectory(find)
+add_subdirectory(connect_cancel)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/CMakeLists.txt
new file mode 100644
index 0000000..9ad919a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/CMakeLists.txt
@@ -0,0 +1,35 @@
+#
+# 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.
+#
+
+# Default LIBHDFSPP_DIR to the default install location.  You can override
+#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
+set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
+
+include_directories( ${LIBHDFSPP_DIR}/include )
+link_directories( ${LIBHDFSPP_DIR}/lib )
+
+add_executable(cat_cc cat.cc)
+target_link_libraries(cat_cc hdfspp)
+
+# Several examples in different languages need to produce executables with
+# same names. To allow executables with same names we keep their CMake
+# names different, but specify their executable names as follows:
+set_target_properties( cat_cc
+    PROPERTIES
+    OUTPUT_NAME "cat"
+)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/cat.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/cat.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/cat.cc
new file mode 100644
index 0000000..bca2a94
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/cat/cat.cc
@@ -0,0 +1,109 @@
+/*
+  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.
+*/
+
+/**
+   * Unix-like cat tool example.
+   *
+   * Reads the specified file from HDFS and outputs to stdout.
+   *
+   * Usage: cat /<path-to-file>
+   *
+   * Example: cat /dir/file
+   *
+   * @param path-to-file    Absolute path to the file to read.
+   *
+   **/
+
+#include "hdfspp/hdfspp.h"
+#include "common/hdfs_configuration.h"
+#include "common/configuration_loader.h"
+
+#include <google/protobuf/stubs/common.h>
+
+const std::size_t BUF_SIZE = 1048576; //1 MB
+static char input_buffer[BUF_SIZE];
+
+int main(int argc, char *argv[]) {
+  if (argc != 2) {
+    std::cerr << "usage: cat /<path-to-file>" << std::endl;
+    exit(EXIT_FAILURE);
+  }
+  std::string path = argv[1];
+
+  hdfs::Options options;
+  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
+  hdfs::ConfigurationLoader loader;
+  //Loading default config files core-site.xml and hdfs-site.xml from the config path
+  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
+  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
+  if(config){
+    //Loading options from the config
+    options = config->GetOptions();
+  }
+  hdfs::IoService * io_service = hdfs::IoService::New();
+  //Wrapping fs into a shared pointer to guarantee deletion
+  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
+  if (!fs) {
+    std::cerr << "Could not connect the file system." << std::endl;
+    exit(EXIT_FAILURE);
+  }
+  hdfs::Status status = fs->ConnectToDefaultFs();
+  if (!status.ok()) {
+    if(!options.defaultFS.get_host().empty()){
+      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
+    } else {
+      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
+    }
+    exit(EXIT_FAILURE);
+  }
+
+  hdfs::FileHandle *file_raw = nullptr;
+  status = fs->Open(path, &file_raw);
+  if (!status.ok()) {
+    std::cerr << "Could not open file " << path << ". " << status.ToString() << std::endl;
+    exit(EXIT_FAILURE);
+  }
+  //wrapping file_raw into a unique pointer to guarantee deletion
+  std::unique_ptr<hdfs::FileHandle> file(file_raw);
+
+  ssize_t total_bytes_read = 0;
+  size_t last_bytes_read = 0;
+
+  do{
+    //Reading file chunks
+    status = file->Read(input_buffer, sizeof(input_buffer), &last_bytes_read);
+    if(status.ok()) {
+      //Writing file chunks to stdout
+      fwrite(input_buffer, last_bytes_read, 1, stdout);
+      total_bytes_read += last_bytes_read;
+    } else {
+      if(status.is_invalid_offset()){
+        //Reached the end of the file
+        break;
+      } else {
+        std::cerr << "Error reading the file: " << status.ToString() << std::endl;
+        exit(EXIT_FAILURE);
+      }
+    }
+  } while (last_bytes_read > 0);
+
+  // Clean up static data and prevent valgrind memory leaks
+  google::protobuf::ShutdownProtobufLibrary();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/CMakeLists.txt
new file mode 100644
index 0000000..6ee44cc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/CMakeLists.txt
@@ -0,0 +1,27 @@
+#
+# 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.
+#
+
+# Default LIBHDFSPP_DIR to the default install location.  You can override
+#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
+set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
+
+include_directories( ${LIBHDFSPP_DIR}/include )
+link_directories( ${LIBHDFSPP_DIR}/lib )
+
+add_executable(connect_cancel connect_cancel.cc)
+target_link_libraries(connect_cancel hdfspp)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/connect_cancel.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/connect_cancel.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/connect_cancel.cc
new file mode 100644
index 0000000..46eef9d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/connect_cancel/connect_cancel.cc
@@ -0,0 +1,154 @@
+/*
+  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 "hdfspp/hdfspp.h"
+#include "common/hdfs_configuration.h"
+#include "common/configuration_loader.h"
+
+#include <google/protobuf/stubs/common.h>
+
+#include <signal.h>
+#include <unistd.h>
+
+#include <thread>
+#include <iostream>
+
+// Simple example of how to cancel an async connect call.
+// Here Control-C (SIGINT) is caught in order to invoke the FS level cancel and
+// properly tear down the process.  Valgrind should show no leaked memory on exit
+// when cancel has been called.  URI parsing code is omitted and defaultFs from
+// /etc/hadoop/conf or $HADOOP_CONF_DIR is always used.
+
+// Scoped globally to make it simple to reference from the signal handler.
+std::shared_ptr<hdfs::FileSystem> fs;
+
+const std::string catch_enter("In signal handler, going to try and cancel FileSystem::Connect.\n");
+const std::string catch_cancel("FileSystem::Cancel has been canceled in the signal handler.\n");
+const std::string catch_exit("Exiting the signal handler.\n");
+
+// Avoid IO reentrancy issues, see comments in signal handler below.
+// It's possible that the write interleaves with another write call,
+// but it won't corrupt the stack or heap.
+static void sighandler_direct_stdout(const std::string &msg) {
+  ssize_t res = ::write(1 /*posix stdout FD*/, msg.data(), msg.size());
+  // In production you'd want to check res, but error handling code will
+  // need to be fairly application specific if it's going to properly
+  // avoid reentrant calls to malloc.
+  (void)res;
+}
+
+// Signal handler to make a SIGINT call cancel rather than exit().
+static void sig_catch(int val) {
+  (void)val;
+  // This is avoiding the tricky bits of signal handling, notably that the
+  // underlying string manipulation and IO functions used by the the logger
+  // are unlikely to be reentrant.
+  //
+  // Production code could mask out all logging on handler entry and enable
+  // it again on exit; here we just assume it's "good enough" and some
+  // (possibly broken) log messages are better than none.
+
+  sighandler_direct_stdout(catch_enter);
+  if(fs) {
+    // This will invoke the callback immediately with an OperationCanceled status
+    fs->CancelPendingConnect();
+    sighandler_direct_stdout(catch_cancel);
+  }
+  sighandler_direct_stdout(catch_exit);
+}
+
+
+int main(int arg_token_count, const char **args) {
+  (void)args;
+  if(arg_token_count != 1) {
+    std::cerr << "usage: ./connect_cancel";
+    google::protobuf::ShutdownProtobufLibrary();
+    exit(EXIT_FAILURE);
+  }
+
+  // Register signal handle to asynchronously invoke cancel from outside the main thread.
+  signal(SIGINT, sig_catch);
+
+  // Generic setup/config code much like the other examples.
+  hdfs::Options options;
+  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
+  hdfs::ConfigurationLoader loader;
+  //Loading default config files core-site.xml and hdfs-site.xml from the config path
+  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
+  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
+  if(config){
+    //Loading options from the config
+    options = config->GetOptions();
+  }
+
+
+  // Start an IoService and some worker threads
+  std::shared_ptr<hdfs::IoService> service = hdfs::IoService::MakeShared();
+  if(nullptr == service) {
+    std::cerr << "Unable to create IoService" << std::endl;
+    fs.reset();
+    // Nasty hack to clean up for valgrind since we don't have the C++17 optional<T>::reset method
+    config = decltype(config)();
+    google::protobuf::ShutdownProtobufLibrary();
+    exit(EXIT_FAILURE);
+  }
+
+  unsigned int worker_count = service->InitDefaultWorkers();
+  if(worker_count < 1) {
+    std::cerr << "Unable to create IoService worker threads";
+    fs.reset();
+    service->Stop();
+    config = decltype(config)();
+    google::protobuf::ShutdownProtobufLibrary();
+    exit(EXIT_FAILURE);
+  }
+
+  // Set up and connect to the FileSystem
+  fs.reset(hdfs::FileSystem::New(service, "", options));
+  if(nullptr == fs) {
+    std::cerr << "Unable to create FileSystem" << std::endl;
+    fs.reset();
+    service->Stop();
+    config = decltype(config)();
+    google::protobuf::ShutdownProtobufLibrary();
+    exit(EXIT_FAILURE);
+  }
+
+  hdfs::Status status = fs->ConnectToDefaultFs();
+  if (!status.ok()) {
+    if(!options.defaultFS.get_host().empty()){
+      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
+    } else {
+      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
+    }
+    fs.reset();
+    service->Stop();
+    config = decltype(config)();
+    google::protobuf::ShutdownProtobufLibrary();
+    exit(EXIT_FAILURE);
+  }
+
+  fs.reset();
+  service->Stop();
+  config = decltype(config)();
+  google::protobuf::ShutdownProtobufLibrary();
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/CMakeLists.txt
new file mode 100644
index 0000000..e894e58
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/CMakeLists.txt
@@ -0,0 +1,35 @@
+#
+# 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.
+#
+
+# Default LIBHDFSPP_DIR to the default install location.  You can override
+#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
+set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
+
+include_directories( ${LIBHDFSPP_DIR}/include )
+link_directories( ${LIBHDFSPP_DIR}/lib )
+
+add_executable(find_cc find.cc)
+target_link_libraries(find_cc hdfspp)
+
+# Several examples in different languages need to produce executables with
+# same names. To allow executables with same names we keep their CMake
+# names different, but specify their executable names as follows:
+set_target_properties( find_cc
+    PROPERTIES
+    OUTPUT_NAME "find"
+)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/find.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/find.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/find.cc
new file mode 100644
index 0000000..21a731b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/find/find.cc
@@ -0,0 +1,162 @@
+/*
+  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.
+*/
+
+/**
+   * A parallel find tool example.
+   *
+   * Finds all files matching the specified name recursively starting from the
+   * specified directory and prints their filepaths. Works either synchronously
+   * or asynchronously.
+   *
+   * Usage: find /<path-to-file> <file-name> <use_async>
+   *
+   * Example: find /dir?/tree* some?file*name 1
+   *
+   * @param path-to-file    Absolute path at which to begin search, can have wild
+   *                        cards and must be non-blank
+   * @param file-name       Name to find, can have wild cards and must be non-blank
+   * @param use_async       If set to 1 it prints out results asynchronously as
+   *                        they arrive. If set to 0 results are printed in one
+   *                        big chunk when it becomes available.
+   *
+   **/
+
+#include "hdfspp/hdfspp.h"
+#include "common/hdfs_configuration.h"
+#include "common/configuration_loader.h"
+
+#include <google/protobuf/stubs/common.h>
+#include <future>
+
+void SyncFind(std::shared_ptr<hdfs::FileSystem> fs, const std::string &path, const std::string &name){
+  std::vector<hdfs::StatInfo> results;
+  //Synchronous call to Find
+  hdfs::Status stat = fs->Find(path, name, hdfs::FileSystem::GetDefaultFindMaxDepth(), &results);
+
+  if (!stat.ok()) {
+    std::cerr << "Error: " << stat.ToString() << std::endl;
+  }
+
+  if(results.empty()){
+    std::cout << "Nothing Found" << std::endl;
+  } else {
+    //Printing out the results
+    for (hdfs::StatInfo const& si : results) {
+      std::cout << si.full_path << std::endl;
+    }
+  }
+}
+
+void AsyncFind(std::shared_ptr<hdfs::FileSystem> fs, const std::string &path, const std::string &name){
+  std::promise<void> promise;
+  std::future<void> future(promise.get_future());
+  bool something_found = false;
+  hdfs::Status status = hdfs::Status::OK();
+
+  /**
+    * Keep requesting more until we get the entire listing. Set the promise
+    * when we have the entire listing to stop.
+    *
+    * Find guarantees that the handler will only be called once at a time,
+    * so we do not need any locking here
+    */
+  auto handler = [&promise, &status, &something_found]
+                  (const hdfs::Status &s, const std::vector<hdfs::StatInfo> & si, bool has_more_results) -> bool {
+    //Print result chunks as they arrive
+    if(!si.empty()) {
+      something_found = true;
+      for (hdfs::StatInfo const& s : si) {
+        std::cout << s.full_path << std::endl;
+      }
+    }
+    if(!s.ok() && status.ok()){
+      //We make sure we set 'status' only on the first error.
+      status = s;
+    }
+    if (!has_more_results) {
+      promise.set_value();  //set promise
+      return false;         //request stop sending results
+    }
+    return true;  //request more results
+  };
+
+  //Asynchronous call to Find
+  fs->Find(path, name, hdfs::FileSystem::GetDefaultFindMaxDepth(), handler);
+
+  //block until promise is set
+  future.get();
+  if(!status.ok()) {
+    std::cerr << "Error: " << status.ToString() << std::endl;
+  }
+  if(!something_found){
+    std::cout << "Nothing Found" << std::endl;
+  }
+}
+
+int main(int argc, char *argv[]) {
+  if (argc != 4) {
+    std::cerr << "usage: find /<path-to-file> <file-name> <use_async>" << std::endl;
+    exit(EXIT_FAILURE);
+  }
+
+  std::string path = argv[1];
+  std::string name = argv[2];
+  bool use_async = (std::stoi(argv[3]) != 0);
+
+  hdfs::Options options;
+  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
+  hdfs::ConfigurationLoader loader;
+  //Loading default config files core-site.xml and hdfs-site.xml from the config path
+  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
+  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
+  if(config){
+    //Loading options from the config
+    options = config->GetOptions();
+  }
+  //TODO: HDFS-9539 - until then we increase the time-out to allow all recursive async calls to finish
+  options.rpc_timeout = std::numeric_limits<int>::max();
+  hdfs::IoService * io_service = hdfs::IoService::New();
+  //Wrapping fs into a unique pointer to guarantee deletion
+  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
+  if (!fs) {
+    std::cerr << "Could not connect the file system." << std::endl;
+    exit(EXIT_FAILURE);
+  }
+  hdfs::Status status = fs->ConnectToDefaultFs();
+  if (!status.ok()) {
+    if(!options.defaultFS.get_host().empty()){
+      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
+    } else {
+      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
+    }
+    exit(EXIT_FAILURE);
+  }
+
+  if (use_async){
+    //Example of Async find
+    AsyncFind(fs, path, name);
+  } else {
+    //Example of Sync find
+    SyncFind(fs, path, name);
+  }
+
+  // Clean up static data and prevent valgrind memory leaks
+  google::protobuf::ShutdownProtobufLibrary();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/CMakeLists.txt
new file mode 100644
index 0000000..e25f425
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/CMakeLists.txt
@@ -0,0 +1,35 @@
+#
+# 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.
+#
+
+# Default LIBHDFSPP_DIR to the default install location.  You can override
+#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
+set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
+
+include_directories( ${LIBHDFSPP_DIR}/include )
+link_directories( ${LIBHDFSPP_DIR}/lib )
+
+add_executable(gendirs_cc gendirs.cc)
+target_link_libraries(gendirs_cc hdfspp)
+
+# Several examples in different languages need to produce executables with
+# same names. To allow executables with same names we keep their CMake
+# names different, but specify their executable names as follows:
+set_target_properties( gendirs_cc
+    PROPERTIES
+    OUTPUT_NAME "gendirs"
+)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/gendirs.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/gendirs.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/gendirs.cc
new file mode 100644
index 0000000..c90abbd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cc/gendirs/gendirs.cc
@@ -0,0 +1,144 @@
+/*
+  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.
+*/
+
+/**
+   * A recursive directory generator tool.
+   *
+   * Generates a directory tree with specified depth and fanout starting from
+   * a given path. Generation is asynchronous.
+   *
+   * Usage:   gendirs /<path-to-dir> <depth> <fanout>
+   *
+   * Example: gendirs /dir0 3 10
+   *
+   * @param path-to-dir   Absolute path to the directory tree root where the
+   *                      directory tree will be generated
+   * @param depth         Depth of the directory tree (number of levels from
+   *                      root to leaves)
+   * @param fanout        Fanout of each directory (number of sub-directories to
+   *                      be created inside each directory except leaf directories)
+   *
+   **/
+
+#include "hdfspp/hdfspp.h"
+#include "common/hdfs_configuration.h"
+#include "common/configuration_loader.h"
+
+#include <google/protobuf/stubs/common.h>
+#include <future>
+
+#define DEFAULT_PERMISSIONS 0755
+
+void GenerateDirectories (std::shared_ptr<hdfs::FileSystem> fs, int depth, int level, int fanout, std::string path, std::vector<std::future<hdfs::Status>> & futures) {
+  //Level contains our current depth in the directory tree
+  if(level < depth) {
+    for(int i = 0; i < fanout; i++){
+      //Recursive calls to cover all possible paths from the root to the leave nodes
+      GenerateDirectories(fs, depth, level+1, fanout, path + "dir" + std::to_string(i) + "/", futures);
+    }
+  } else {
+    //We have reached the leaf nodes and now start making calls to create directories
+    //We make a promise which will be set when the call finishes and executes our handler
+    auto callstate = std::make_shared<std::promise<hdfs::Status>>();
+    //Extract a future from this promise
+    std::future<hdfs::Status> future(callstate->get_future());
+    //Save this future to the vector of futures which will be used to wait on all promises
+    //after the whole recursion is done
+    futures.push_back(std::move(future));
+    //Create a handler that will be executed when Mkdirs is done
+    auto handler = [callstate](const hdfs::Status &s) {
+      callstate->set_value(s);
+    };
+    //Asynchronous call to create this directory along with all missing parent directories
+    fs->Mkdirs(path, DEFAULT_PERMISSIONS, true, handler);
+  }
+}
+
+int main(int argc, char *argv[]) {
+  if (argc != 4) {
+    std::cerr << "usage: gendirs /<path-to-dir> <depth> <fanout>" << std::endl;
+    exit(EXIT_FAILURE);
+  }
+
+  std::string path = argv[1];
+  int depth = std::stoi(argv[2]);
+  int fanout = std::stoi(argv[3]);
+
+  hdfs::Options options;
+  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
+  hdfs::ConfigurationLoader loader;
+  //Loading default config files core-site.xml and hdfs-site.xml from the config path
+  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
+  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
+  if(config){
+    //Loading options from the config
+    options = config->GetOptions();
+  }
+  //TODO: HDFS-9539 - until then we increase the time-out to allow all recursive async calls to finish
+  options.rpc_timeout = std::numeric_limits<int>::max();
+  hdfs::IoService * io_service = hdfs::IoService::New();
+  //Wrapping fs into a unique pointer to guarantee deletion
+  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
+  if (!fs) {
+    std::cerr << "Could not connect the file system." << std::endl;
+    exit(EXIT_FAILURE);
+  }
+  hdfs::Status status = fs->ConnectToDefaultFs();
+  if (!status.ok()) {
+    if(!options.defaultFS.get_host().empty()){
+      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
+    } else {
+      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
+    }
+    exit(EXIT_FAILURE);
+  }
+
+  /**
+   * We do not want the recursion to block on anything, therefore we will be
+   * making asynchronous calls recursively, and then just waiting for all
+   * the calls to finish.
+   *
+   * This array of futures will be populated by the recursive function below.
+   * Each new asynchronous Mkdirs call will add a future to this vector, and will
+   * create a promise, which will only be set when the call was completed and
+   * processed. After the whole recursion is complete we will need to wait until
+   * all promises are set before we can exit.
+   **/
+  std::vector<std::future<hdfs::Status>> futures;
+
+  GenerateDirectories(fs, depth, 0, fanout, path + "/", futures);
+
+  /**
+   * We are waiting here until all promises are set, and checking whether
+   * the returned statuses contained any errors.
+   **/
+  for(std::future<hdfs::Status> &fs : futures){
+    hdfs::Status status = fs.get();
+    if (!status.ok()) {
+      std::cerr << "Error: " << status.ToString() << std::endl;
+      exit(EXIT_FAILURE);
+    }
+  }
+
+  std::cout << "All done!" << std::endl;
+
+  // Clean up static data and prevent valgrind memory leaks
+  google::protobuf::ShutdownProtobufLibrary();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/CMakeLists.txt
deleted file mode 100644
index 8893f03..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/CMakeLists.txt
+++ /dev/null
@@ -1,22 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-add_subdirectory(cat)
-add_subdirectory(gendirs)
-add_subdirectory(find)
-add_subdirectory(connect_cancel)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/CMakeLists.txt
deleted file mode 100644
index 9ec3332..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/CMakeLists.txt
+++ /dev/null
@@ -1,35 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# Default LIBHDFSPP_DIR to the default install location.  You can override
-#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
-set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
-
-include_directories( ${LIBHDFSPP_DIR}/include )
-link_directories( ${LIBHDFSPP_DIR}/lib )
-
-add_executable(cat_cpp cat.cpp)
-target_link_libraries(cat_cpp hdfspp)
-
-# Several examples in different languages need to produce executables with
-# same names. To allow executables with same names we keep their CMake
-# names different, but specify their executable names as follows:
-set_target_properties( cat_cpp
-    PROPERTIES
-    OUTPUT_NAME "cat"
-)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/cat.cpp
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/cat.cpp b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/cat.cpp
deleted file mode 100644
index 17626ea..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/cat/cat.cpp
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing,
-  software distributed under the License is distributed on an
-  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-  KIND, either express or implied.  See the License for the
-  specific language governing permissions and limitations
-  under the License.
-*/
-
-/**
-   * Unix-like cat tool example.
-   *
-   * Reads the specified file from HDFS and outputs to stdout.
-   *
-   * Usage: cat /<path-to-file>
-   *
-   * Example: cat /dir/file
-   *
-   * @param path-to-file    Absolute path to the file to read.
-   *
-   **/
-
-#include "hdfspp/hdfspp.h"
-#include "common/hdfs_configuration.h"
-#include "common/configuration_loader.h"
-
-#include <google/protobuf/stubs/common.h>
-
-#define BUF_SIZE 4096
-
-int main(int argc, char *argv[]) {
-  if (argc != 2) {
-    std::cerr << "usage: cat /<path-to-file>" << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  std::string path = argv[1];
-
-  hdfs::Options options;
-  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
-  hdfs::ConfigurationLoader loader;
-  //Loading default config files core-site.xml and hdfs-site.xml from the config path
-  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
-  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
-  if(config){
-    //Loading options from the config
-    options = config->GetOptions();
-  }
-  hdfs::IoService * io_service = hdfs::IoService::New();
-  //Wrapping fs into a shared pointer to guarantee deletion
-  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
-  if (!fs) {
-    std::cerr << "Could not connect the file system." << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  hdfs::Status status = fs->ConnectToDefaultFs();
-  if (!status.ok()) {
-    if(!options.defaultFS.get_host().empty()){
-      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
-    } else {
-      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
-    }
-    exit(EXIT_FAILURE);
-  }
-
-  hdfs::FileHandle *file_raw = nullptr;
-  status = fs->Open(path, &file_raw);
-  if (!status.ok()) {
-    std::cerr << "Could not open file " << path << ". " << status.ToString() << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  //wrapping file_raw into a unique pointer to guarantee deletion
-  std::unique_ptr<hdfs::FileHandle> file(file_raw);
-
-  char input_buffer[BUF_SIZE];
-  ssize_t total_bytes_read = 0;
-  size_t last_bytes_read = 0;
-
-  do{
-    //Reading file chunks
-    status = file->Read(input_buffer, sizeof(input_buffer), &last_bytes_read);
-    if(status.ok()) {
-      //Writing file chunks to stdout
-      fwrite(input_buffer, last_bytes_read, 1, stdout);
-      total_bytes_read += last_bytes_read;
-    } else {
-      if(status.is_invalid_offset()){
-        //Reached the end of the file
-        break;
-      } else {
-        std::cerr << "Error reading the file: " << status.ToString() << std::endl;
-        exit(EXIT_FAILURE);
-      }
-    }
-  } while (last_bytes_read > 0);
-
-  // Clean up static data and prevent valgrind memory leaks
-  google::protobuf::ShutdownProtobufLibrary();
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/CMakeLists.txt
deleted file mode 100644
index a029a99..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/CMakeLists.txt
+++ /dev/null
@@ -1,29 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# Default LIBHDFSPP_DIR to the default install location.  You can override
-#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
-set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
-
-include_directories( ${LIBHDFSPP_DIR}/include )
-link_directories( ${LIBHDFSPP_DIR}/lib )
-
-add_executable(connect_cancel connect_cancel.cc)
-target_link_libraries(connect_cancel hdfspp)
-
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/connect_cancel.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/connect_cancel.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/connect_cancel.cc
deleted file mode 100644
index 7e5daa9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/connect_cancel/connect_cancel.cc
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing,
-  software distributed under the License is distributed on an
-  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-  KIND, either express or implied.  See the License for the
-  specific language governing permissions and limitations
-  under the License.
-*/
-
-
-#include "hdfspp/hdfspp.h"
-#include "common/hdfs_configuration.h"
-#include "common/configuration_loader.h"
-
-#include <google/protobuf/stubs/common.h>
-
-#include <signal.h>
-#include <unistd.h>
-
-#include <thread>
-#include <iostream>
-
-// Simple example of how to cancel an async connect call.
-// Here Control-C (SIGINT) is caught in order to invoke the FS level cancel and
-// properly tear down the process.  Valgrind should show no leaked memory on exit
-// when cancel has been called.  URI parsing code is omitted and defaultFs from
-// /etc/hadoop/conf or $HADOOP_CONF_DIR is always used.
-
-// Scoped globally to make it simple to reference from the signal handler.
-std::shared_ptr<hdfs::FileSystem> fs;
-
-const std::string catch_enter("In signal handler, going to try and cancel FileSystem::Connect.\n");
-const std::string catch_cancel("FileSystem::Cancel has been canceled in the signal handler.\n");
-const std::string catch_exit("Exiting the signal handler.\n");
-
-// Avoid IO reentrancy issues, see comments in signal handler below.
-// It's possible that the write interleaves with another write call,
-// but it won't corrupt the stack or heap.
-static void sighandler_direct_stdout(const std::string &msg) {
-  ssize_t res = ::write(1 /*posix stdout FD*/, msg.data(), msg.size());
-  // In production you'd want to check res, but error handling code will
-  // need to be fairly application specific if it's going to properly
-  // avoid reentrant calls to malloc.
-  (void)res;
-}
-
-// Signal handler to make a SIGINT call cancel rather than exit().
-static void sig_catch(int val) {
-  (void)val;
-  // This is avoiding the tricky bits of signal handling, notably that the
-  // underlying string manipulation and IO functions used by the the logger
-  // are unlikely to be reentrant.
-  //
-  // Production code could mask out all logging on handler entry and enable
-  // it again on exit; here we just assume it's "good enough" and some
-  // (possibly broken) log messages are better than none.
-
-  sighandler_direct_stdout(catch_enter);
-  if(fs) {
-    // This will invoke the callback immediately with an OperationCanceled status
-    fs->CancelPendingConnect();
-    sighandler_direct_stdout(catch_cancel);
-  }
-  sighandler_direct_stdout(catch_exit);
-}
-
-
-int main(int arg_token_count, const char **args) {
-  (void)args;
-  if(arg_token_count != 1) {
-    std::cerr << "usage: ./connect_cancel";
-    google::protobuf::ShutdownProtobufLibrary();
-    exit(EXIT_FAILURE);
-  }
-
-  // Register signal handle to asynchronously invoke cancel from outside the main thread.
-  signal(SIGINT, sig_catch);
-
-  // Generic setup/config code much like the other examples.
-  hdfs::Options options;
-  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
-  hdfs::ConfigurationLoader loader;
-  //Loading default config files core-site.xml and hdfs-site.xml from the config path
-  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
-  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
-  if(config){
-    //Loading options from the config
-    options = config->GetOptions();
-  }
-
-
-  // Start an IoService and some worker threads
-  std::shared_ptr<hdfs::IoService> service = hdfs::IoService::MakeShared();
-  if(nullptr == service) {
-    std::cerr << "Unable to create IoService" << std::endl;
-    fs.reset();
-    // Nasty hack to clean up for valgrind since we don't have the C++17 optional<T>::reset method
-    config = decltype(config)();
-    google::protobuf::ShutdownProtobufLibrary();
-    exit(EXIT_FAILURE);
-  }
-
-  unsigned int worker_count = service->InitDefaultWorkers();
-  if(worker_count < 1) {
-    std::cerr << "Unable to create IoService worker threads";
-    fs.reset();
-    service->Stop();
-    config = decltype(config)();
-    google::protobuf::ShutdownProtobufLibrary();
-    exit(EXIT_FAILURE);
-  }
-
-  // Set up and connect to the FileSystem
-  fs.reset(hdfs::FileSystem::New(service, "", options));
-  if(nullptr == fs) {
-    std::cerr << "Unable to create FileSystem" << std::endl;
-    fs.reset();
-    service->Stop();
-    config = decltype(config)();
-    google::protobuf::ShutdownProtobufLibrary();
-    exit(EXIT_FAILURE);
-  }
-
-  hdfs::Status status = fs->ConnectToDefaultFs();
-  if (!status.ok()) {
-    if(!options.defaultFS.get_host().empty()){
-      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
-    } else {
-      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
-    }
-    fs.reset();
-    service->Stop();
-    config = decltype(config)();
-    google::protobuf::ShutdownProtobufLibrary();
-    exit(EXIT_FAILURE);
-  }
-
-  fs.reset();
-  service->Stop();
-  config = decltype(config)();
-  google::protobuf::ShutdownProtobufLibrary();
-
-  return 0;
-}
-
-
-
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/CMakeLists.txt
deleted file mode 100644
index 7ae27c5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/CMakeLists.txt
+++ /dev/null
@@ -1,35 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# Default LIBHDFSPP_DIR to the default install location.  You can override
-#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
-set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
-
-include_directories( ${LIBHDFSPP_DIR}/include )
-link_directories( ${LIBHDFSPP_DIR}/lib )
-
-add_executable(find_cpp find.cpp)
-target_link_libraries(find_cpp hdfspp)
-
-# Several examples in different languages need to produce executables with
-# same names. To allow executables with same names we keep their CMake
-# names different, but specify their executable names as follows:
-set_target_properties( find_cpp
-    PROPERTIES
-    OUTPUT_NAME "find"
-)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/find.cpp
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/find.cpp b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/find.cpp
deleted file mode 100644
index 21a731b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/find/find.cpp
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing,
-  software distributed under the License is distributed on an
-  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-  KIND, either express or implied.  See the License for the
-  specific language governing permissions and limitations
-  under the License.
-*/
-
-/**
-   * A parallel find tool example.
-   *
-   * Finds all files matching the specified name recursively starting from the
-   * specified directory and prints their filepaths. Works either synchronously
-   * or asynchronously.
-   *
-   * Usage: find /<path-to-file> <file-name> <use_async>
-   *
-   * Example: find /dir?/tree* some?file*name 1
-   *
-   * @param path-to-file    Absolute path at which to begin search, can have wild
-   *                        cards and must be non-blank
-   * @param file-name       Name to find, can have wild cards and must be non-blank
-   * @param use_async       If set to 1 it prints out results asynchronously as
-   *                        they arrive. If set to 0 results are printed in one
-   *                        big chunk when it becomes available.
-   *
-   **/
-
-#include "hdfspp/hdfspp.h"
-#include "common/hdfs_configuration.h"
-#include "common/configuration_loader.h"
-
-#include <google/protobuf/stubs/common.h>
-#include <future>
-
-void SyncFind(std::shared_ptr<hdfs::FileSystem> fs, const std::string &path, const std::string &name){
-  std::vector<hdfs::StatInfo> results;
-  //Synchronous call to Find
-  hdfs::Status stat = fs->Find(path, name, hdfs::FileSystem::GetDefaultFindMaxDepth(), &results);
-
-  if (!stat.ok()) {
-    std::cerr << "Error: " << stat.ToString() << std::endl;
-  }
-
-  if(results.empty()){
-    std::cout << "Nothing Found" << std::endl;
-  } else {
-    //Printing out the results
-    for (hdfs::StatInfo const& si : results) {
-      std::cout << si.full_path << std::endl;
-    }
-  }
-}
-
-void AsyncFind(std::shared_ptr<hdfs::FileSystem> fs, const std::string &path, const std::string &name){
-  std::promise<void> promise;
-  std::future<void> future(promise.get_future());
-  bool something_found = false;
-  hdfs::Status status = hdfs::Status::OK();
-
-  /**
-    * Keep requesting more until we get the entire listing. Set the promise
-    * when we have the entire listing to stop.
-    *
-    * Find guarantees that the handler will only be called once at a time,
-    * so we do not need any locking here
-    */
-  auto handler = [&promise, &status, &something_found]
-                  (const hdfs::Status &s, const std::vector<hdfs::StatInfo> & si, bool has_more_results) -> bool {
-    //Print result chunks as they arrive
-    if(!si.empty()) {
-      something_found = true;
-      for (hdfs::StatInfo const& s : si) {
-        std::cout << s.full_path << std::endl;
-      }
-    }
-    if(!s.ok() && status.ok()){
-      //We make sure we set 'status' only on the first error.
-      status = s;
-    }
-    if (!has_more_results) {
-      promise.set_value();  //set promise
-      return false;         //request stop sending results
-    }
-    return true;  //request more results
-  };
-
-  //Asynchronous call to Find
-  fs->Find(path, name, hdfs::FileSystem::GetDefaultFindMaxDepth(), handler);
-
-  //block until promise is set
-  future.get();
-  if(!status.ok()) {
-    std::cerr << "Error: " << status.ToString() << std::endl;
-  }
-  if(!something_found){
-    std::cout << "Nothing Found" << std::endl;
-  }
-}
-
-int main(int argc, char *argv[]) {
-  if (argc != 4) {
-    std::cerr << "usage: find /<path-to-file> <file-name> <use_async>" << std::endl;
-    exit(EXIT_FAILURE);
-  }
-
-  std::string path = argv[1];
-  std::string name = argv[2];
-  bool use_async = (std::stoi(argv[3]) != 0);
-
-  hdfs::Options options;
-  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
-  hdfs::ConfigurationLoader loader;
-  //Loading default config files core-site.xml and hdfs-site.xml from the config path
-  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
-  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
-  if(config){
-    //Loading options from the config
-    options = config->GetOptions();
-  }
-  //TODO: HDFS-9539 - until then we increase the time-out to allow all recursive async calls to finish
-  options.rpc_timeout = std::numeric_limits<int>::max();
-  hdfs::IoService * io_service = hdfs::IoService::New();
-  //Wrapping fs into a unique pointer to guarantee deletion
-  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
-  if (!fs) {
-    std::cerr << "Could not connect the file system." << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  hdfs::Status status = fs->ConnectToDefaultFs();
-  if (!status.ok()) {
-    if(!options.defaultFS.get_host().empty()){
-      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
-    } else {
-      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
-    }
-    exit(EXIT_FAILURE);
-  }
-
-  if (use_async){
-    //Example of Async find
-    AsyncFind(fs, path, name);
-  } else {
-    //Example of Sync find
-    SyncFind(fs, path, name);
-  }
-
-  // Clean up static data and prevent valgrind memory leaks
-  google::protobuf::ShutdownProtobufLibrary();
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/CMakeLists.txt
deleted file mode 100644
index 90cddf0..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/CMakeLists.txt
+++ /dev/null
@@ -1,35 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-# Default LIBHDFSPP_DIR to the default install location.  You can override
-#    it by add -DLIBHDFSPP_DIR=... to your cmake invocation
-set(LIBHDFSPP_DIR CACHE STRING ${CMAKE_INSTALL_PREFIX})
-
-include_directories( ${LIBHDFSPP_DIR}/include )
-link_directories( ${LIBHDFSPP_DIR}/lib )
-
-add_executable(gendirs_cpp gendirs.cpp)
-target_link_libraries(gendirs_cpp hdfspp)
-
-# Several examples in different languages need to produce executables with
-# same names. To allow executables with same names we keep their CMake
-# names different, but specify their executable names as follows:
-set_target_properties( gendirs_cpp
-    PROPERTIES
-    OUTPUT_NAME "gendirs"
-)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/gendirs.cpp
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/gendirs.cpp b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/gendirs.cpp
deleted file mode 100644
index c90abbd..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/examples/cpp/gendirs/gendirs.cpp
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing,
-  software distributed under the License is distributed on an
-  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-  KIND, either express or implied.  See the License for the
-  specific language governing permissions and limitations
-  under the License.
-*/
-
-/**
-   * A recursive directory generator tool.
-   *
-   * Generates a directory tree with specified depth and fanout starting from
-   * a given path. Generation is asynchronous.
-   *
-   * Usage:   gendirs /<path-to-dir> <depth> <fanout>
-   *
-   * Example: gendirs /dir0 3 10
-   *
-   * @param path-to-dir   Absolute path to the directory tree root where the
-   *                      directory tree will be generated
-   * @param depth         Depth of the directory tree (number of levels from
-   *                      root to leaves)
-   * @param fanout        Fanout of each directory (number of sub-directories to
-   *                      be created inside each directory except leaf directories)
-   *
-   **/
-
-#include "hdfspp/hdfspp.h"
-#include "common/hdfs_configuration.h"
-#include "common/configuration_loader.h"
-
-#include <google/protobuf/stubs/common.h>
-#include <future>
-
-#define DEFAULT_PERMISSIONS 0755
-
-void GenerateDirectories (std::shared_ptr<hdfs::FileSystem> fs, int depth, int level, int fanout, std::string path, std::vector<std::future<hdfs::Status>> & futures) {
-  //Level contains our current depth in the directory tree
-  if(level < depth) {
-    for(int i = 0; i < fanout; i++){
-      //Recursive calls to cover all possible paths from the root to the leave nodes
-      GenerateDirectories(fs, depth, level+1, fanout, path + "dir" + std::to_string(i) + "/", futures);
-    }
-  } else {
-    //We have reached the leaf nodes and now start making calls to create directories
-    //We make a promise which will be set when the call finishes and executes our handler
-    auto callstate = std::make_shared<std::promise<hdfs::Status>>();
-    //Extract a future from this promise
-    std::future<hdfs::Status> future(callstate->get_future());
-    //Save this future to the vector of futures which will be used to wait on all promises
-    //after the whole recursion is done
-    futures.push_back(std::move(future));
-    //Create a handler that will be executed when Mkdirs is done
-    auto handler = [callstate](const hdfs::Status &s) {
-      callstate->set_value(s);
-    };
-    //Asynchronous call to create this directory along with all missing parent directories
-    fs->Mkdirs(path, DEFAULT_PERMISSIONS, true, handler);
-  }
-}
-
-int main(int argc, char *argv[]) {
-  if (argc != 4) {
-    std::cerr << "usage: gendirs /<path-to-dir> <depth> <fanout>" << std::endl;
-    exit(EXIT_FAILURE);
-  }
-
-  std::string path = argv[1];
-  int depth = std::stoi(argv[2]);
-  int fanout = std::stoi(argv[3]);
-
-  hdfs::Options options;
-  //Setting the config path to the default: "$HADOOP_CONF_DIR" or "/etc/hadoop/conf"
-  hdfs::ConfigurationLoader loader;
-  //Loading default config files core-site.xml and hdfs-site.xml from the config path
-  hdfs::optional<hdfs::HdfsConfiguration> config = loader.LoadDefaultResources<hdfs::HdfsConfiguration>();
-  //TODO: HDFS-9539 - after this is resolved, valid config will always be returned.
-  if(config){
-    //Loading options from the config
-    options = config->GetOptions();
-  }
-  //TODO: HDFS-9539 - until then we increase the time-out to allow all recursive async calls to finish
-  options.rpc_timeout = std::numeric_limits<int>::max();
-  hdfs::IoService * io_service = hdfs::IoService::New();
-  //Wrapping fs into a unique pointer to guarantee deletion
-  std::shared_ptr<hdfs::FileSystem> fs(hdfs::FileSystem::New(io_service, "", options));
-  if (!fs) {
-    std::cerr << "Could not connect the file system." << std::endl;
-    exit(EXIT_FAILURE);
-  }
-  hdfs::Status status = fs->ConnectToDefaultFs();
-  if (!status.ok()) {
-    if(!options.defaultFS.get_host().empty()){
-      std::cerr << "Error connecting to " << options.defaultFS << ". " << status.ToString() << std::endl;
-    } else {
-      std::cerr << "Error connecting to the cluster: defaultFS is empty. " << status.ToString() << std::endl;
-    }
-    exit(EXIT_FAILURE);
-  }
-
-  /**
-   * We do not want the recursion to block on anything, therefore we will be
-   * making asynchronous calls recursively, and then just waiting for all
-   * the calls to finish.
-   *
-   * This array of futures will be populated by the recursive function below.
-   * Each new asynchronous Mkdirs call will add a future to this vector, and will
-   * create a promise, which will only be set when the call was completed and
-   * processed. After the whole recursion is complete we will need to wait until
-   * all promises are set before we can exit.
-   **/
-  std::vector<std::future<hdfs::Status>> futures;
-
-  GenerateDirectories(fs, depth, 0, fanout, path + "/", futures);
-
-  /**
-   * We are waiting here until all promises are set, and checking whether
-   * the returned statuses contained any errors.
-   **/
-  for(std::future<hdfs::Status> &fs : futures){
-    hdfs::Status status = fs.get();
-    if (!status.ok()) {
-      std::cerr << "Error: " << status.ToString() << std::endl;
-      exit(EXIT_FAILURE);
-    }
-  }
-
-  std::cout << "All done!" << std::endl;
-
-  // Clean up static data and prevent valgrind memory leaks
-  google::protobuf::ShutdownProtobufLibrary();
-  return 0;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/content_summary.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/content_summary.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/content_summary.h
new file mode 100644
index 0000000..6762a65
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/content_summary.h
@@ -0,0 +1,48 @@
+/**
+ * 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 HDFSPP_CONTENT_SUMMARY_H_
+#define HDFSPP_CONTENT_SUMMARY_H_
+
+#include <string>
+
+namespace hdfs {
+
+/**
+ * Content summary is assumed to be unchanging for the duration of the operation
+ */
+struct ContentSummary {
+  uint64_t length;
+  uint64_t filecount;
+  uint64_t directorycount;
+  uint64_t quota;
+  uint64_t spaceconsumed;
+  uint64_t spacequota;
+  std::string path;
+
+  ContentSummary();
+
+  //Converts ContentSummary object to std::string (hdfs_count format)
+  std::string str(bool include_quota) const;
+
+  //Converts ContentSummary object to std::string (hdfs_du format)
+  std::string str_du() const;
+};
+
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/fsinfo.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/fsinfo.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/fsinfo.h
index bd58592..103cf13 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/fsinfo.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/fsinfo.h
@@ -18,6 +18,8 @@
 #ifndef HDFSPP_FSINFO_H_
 #define HDFSPP_FSINFO_H_
 
+#include <string>
+
 namespace hdfs {
 
 /**
@@ -26,25 +28,19 @@ namespace hdfs {
  */
 struct FsInfo {
 
-  unsigned long int     capacity;
-  unsigned long int     used;
-  unsigned long int     remaining;
-  unsigned long int     under_replicated;
-  unsigned long int     corrupt_blocks;
-  unsigned long int     missing_blocks;
-  unsigned long int     missing_repl_one_blocks;
-  unsigned long int     blocks_in_future;
-
-  FsInfo()
-      : capacity(0),
-        used(0),
-        remaining(0),
-        under_replicated(0),
-        corrupt_blocks(0),
-        missing_blocks(0),
-        missing_repl_one_blocks(0),
-        blocks_in_future(0) {
-  }
+  uint64_t capacity;
+  uint64_t used;
+  uint64_t remaining;
+  uint64_t under_replicated;
+  uint64_t corrupt_blocks;
+  uint64_t missing_blocks;
+  uint64_t missing_repl_one_blocks;
+  uint64_t blocks_in_future;
+
+  FsInfo();
+
+  //Converts FsInfo object to std::string (hdfs_df format)
+  std::string str(const std::string fs_name) const;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/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 2b15cf4..cdd3d4d 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
@@ -330,6 +330,17 @@ LIBHDFS_EXTERNAL
 int hdfsDeleteSnapshot(hdfsFS fs, const char* path, const char* name);
 
 /**
+ * Renames the directory snapshot specified by path from old_name to new_name
+ *
+ *  @param fs         The filesystem (required)
+ *  @param path       Path to the snapshotted directory (must be non-blank)
+ *  @param old_name   Current name of the snapshot (must be non-blank)
+ *  @param new_name   New name of the snapshot (must be non-blank)
+ *  @return           0 on success, corresponding errno on failure
+ **/
+int hdfsRenameSnapshot(hdfsFS fs, const char* path, const char* old_name, const char* new_name);
+
+/**
  * Allows snapshots to be made on the specified directory
  *
  *  @param fs      The filesystem (required)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/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 46a1e61..d193b5f 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
@@ -24,6 +24,7 @@
 #include "hdfspp/block_location.h"
 #include "hdfspp/statinfo.h"
 #include "hdfspp/fsinfo.h"
+#include "hdfspp/content_summary.h"
 
 #include <functional>
 #include <memory>
@@ -276,6 +277,14 @@ class FileSystem {
   virtual Status GetFileInfo(const std::string &path, StatInfo & stat_info) = 0;
 
   /**
+   * Returns the number of directories, files and bytes under the given path
+   **/
+  virtual void
+  GetContentSummary(const std::string &path,
+                  const std::function<void(const Status &, const ContentSummary &)> &handler) = 0;
+  virtual Status GetContentSummary(const std::string &path, ContentSummary & stat_info) = 0;
+
+  /**
    * Retrieves the file system information as a whole, such as the total raw size of all files in the filesystem
    * and the raw capacity of the filesystem
    *
@@ -305,7 +314,7 @@ class FileSystem {
 
   /**
    * Returns the locations of all known blocks for the indicated file (or part of it), or an error
-   * if the information clould not be found
+   * if the information could not be found
    */
   virtual void GetBlockLocations(const std::string & path, uint64_t offset, uint64_t length,
     const std::function<void(const Status &, std::shared_ptr<FileBlockLocation> locations)> ) = 0;
@@ -418,6 +427,18 @@ class FileSystem {
       const std::string &name) = 0;
 
   /**
+   * Renames the directory snapshot specified by path from old_name to new_name
+   *
+   *  @param path       Path to the snapshotted directory (must be non-blank)
+   *  @param old_name   Current name of the snapshot (must be non-blank)
+   *  @param new_name   New name of the snapshot (must be non-blank)
+   **/
+  virtual void RenameSnapshot(const std::string &path, const std::string &old_name,
+      const std::string &new_name, const std::function<void(const Status &)> &handler) = 0;
+  virtual Status RenameSnapshot(const std::string &path, const std::string &old_name,
+      const std::string &new_name) = 0;
+
+  /**
    * Allows snapshots to be made on the specified directory
    *
    *  @param path    Path to the directory to be made snapshottable (must be non-empty)
@@ -454,6 +475,8 @@ class FileSystem {
   virtual void SetFsEventCallback(fs_event_callback callback) = 0;
 
   virtual Options get_options() = 0;
+
+  virtual std::string get_cluster_name() = 0;
 };
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/statinfo.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/statinfo.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/statinfo.h
index e077dda..1da1f18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/statinfo.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/statinfo.h
@@ -18,6 +18,8 @@
 #ifndef HDFSPP_STATINFO_H_
 #define HDFSPP_STATINFO_H_
 
+#include <string>
+
 namespace hdfs {
 
 /**
@@ -31,31 +33,25 @@ struct StatInfo {
     IS_SYMLINK = 3
   };
 
-  int                   file_type;
-  ::std::string         path;
-  ::std::string         full_path;
-  unsigned long int     length;
-  unsigned long int     permissions;  //Octal number as in POSIX permissions; e.g. 0777
-  ::std::string         owner;
-  ::std::string         group;
-  unsigned long int     modification_time;
-  unsigned long int     access_time;
-  ::std::string         symlink;
-  unsigned int          block_replication;
-  unsigned long int     blocksize;
-  unsigned long int     fileid;
-  unsigned long int     children_num;
-  StatInfo()
-      : file_type(0),
-        length(0),
-        permissions(0),
-        modification_time(0),
-        access_time(0),
-        block_replication(0),
-        blocksize(0),
-        fileid(0),
-        children_num(0) {
-  }
+  int          file_type;
+  std::string  path;
+  std::string  full_path;
+  uint64_t     length;
+  uint64_t     permissions;  //Octal number as in POSIX permissions; e.g. 0777
+  std::string  owner;
+  std::string  group;
+  uint64_t     modification_time;
+  uint64_t     access_time;
+  std::string  symlink;
+  uint32_t     block_replication;
+  uint64_t     blocksize;
+  uint64_t     fileid;
+  uint64_t     children_num;
+
+  StatInfo();
+
+  //Converts StatInfo object to std::string (hdfs_ls format)
+  std::string str() const;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/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 8c7e34c..7e895b1 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
@@ -1094,6 +1094,37 @@ int hdfsDeleteSnapshot(hdfsFS fs, const char* path, const char* name) {
   }
 }
 
+
+int hdfsRenameSnapshot(hdfsFS fs, const char* path, const char* old_name, const char* new_name) {
+  try {
+    errno = 0;
+    if (!CheckSystem(fs)) {
+      return -1;
+    }
+    const optional<std::string> abs_path = getAbsolutePath(fs, path);
+    if(!abs_path) {
+      return -1;
+    }
+    if (!old_name) {
+      return Error(Status::InvalidArgument("hdfsRenameSnapshot: argument 'old_name' cannot be NULL"));
+    }
+    if (!new_name) {
+      return Error(Status::InvalidArgument("hdfsRenameSnapshot: argument 'new_name' cannot be NULL"));
+    }
+    Status stat;
+    stat = fs->get_impl()->RenameSnapshot(*abs_path, old_name, new_name);
+    if (!stat.ok()) {
+      return Error(stat);
+    }
+    return 0;
+  } catch (const std::exception & e) {
+    return ReportException(e);
+  } catch (...) {
+    return ReportCaughtNonException();
+  }
+
+}
+
 int hdfsAllowSnapshot(hdfsFS fs, const char* path) {
   try {
     errno = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40e3290b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
index bdeb068..b0b721a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
@@ -19,6 +19,6 @@ if(NEED_LINK_DL)
    set(LIB_DL dl)
 endif()
 
-add_library(common_obj OBJECT status.cc sasl_digest_md5.cc hdfs_ioservice.cc options.cc configuration.cc configuration_loader.cc hdfs_configuration.cc uri.cc util.cc retry_policy.cc cancel_tracker.cc logging.cc libhdfs_events_impl.cc auth_info.cc namenode_info.cc)
+add_library(common_obj OBJECT status.cc sasl_digest_md5.cc hdfs_ioservice.cc options.cc configuration.cc configuration_loader.cc hdfs_configuration.cc uri.cc util.cc retry_policy.cc cancel_tracker.cc logging.cc libhdfs_events_impl.cc auth_info.cc namenode_info.cc statinfo.cc fsinfo.cc content_summary.cc)
 add_library(common $<TARGET_OBJECTS:common_obj> $<TARGET_OBJECTS:uriparser2_obj>)
 target_link_libraries(common ${LIB_DL})


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