浏览代码

HDFS-13403: libhdfs++ Use hdfs::IoService object rather than asio::io_service. Contributed by James Clampffer.

James Clampffer 7 年之前
父节点
当前提交
eefe2a147c
共有 36 个文件被更改,包括 476 次插入349 次删除
  1. 1 52
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
  2. 140 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/ioservice.h
  3. 3 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/CMakeLists.txt
  5. 8 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h
  6. 0 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
  7. 18 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.cc
  8. 12 15
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.h
  9. 0 3
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h
  10. 7 8
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
  11. 6 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.h
  12. 13 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc
  13. 9 16
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
  14. 22 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc
  15. 7 19
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h
  16. 9 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc
  17. 5 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h
  18. 58 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
  19. 12 54
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
  21. 9 9
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc
  22. 5 5
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.cc
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.h
  26. 3 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h
  28. 19 13
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc
  29. 5 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h
  30. 7 7
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc
  31. 5 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
  32. 16 15
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc
  33. 6 4
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc
  34. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h
  35. 2 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc
  36. 58 54
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc

+ 1 - 52
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h

@@ -19,6 +19,7 @@
 #define LIBHDFSPP_HDFSPP_H_
 
 #include "hdfspp/options.h"
+#include "hdfspp/ioservice.h"
 #include "hdfspp/status.h"
 #include "hdfspp/events.h"
 #include "hdfspp/block_location.h"
@@ -31,61 +32,9 @@
 
 #include <functional>
 #include <memory>
-#include <set>
-#include <iostream>
 
 namespace hdfs {
 
-/**
- * An IoService manages a queue of asynchronous tasks. All libhdfs++
- * operations are filed against a particular IoService.
- *
- * When an operation is queued into an IoService, the IoService will
- * run the callback handler associated with the operation. Note that
- * the IoService must be stopped before destructing the objects that
- * post the operations.
- *
- * From an implementation point of view the hdfs::IoService provides
- * a thin wrapper over an asio::io_service object so that additional
- * instrumentation and functionality can be added.
- **/
-
-class IoService : public std::enable_shared_from_this<IoService>
-{
- public:
-  static IoService *New();
-  static std::shared_ptr<IoService> MakeShared();
-  virtual ~IoService();
-
-  /**
-   * Start up as many threads as there are logical processors.
-   * Return number of threads created.
-   **/
-  virtual unsigned int InitDefaultWorkers() = 0;
-
-  /**
-   * Initialize with thread_count handler threads.
-   * If thread count is less than one print a log message and default to one thread.
-   * Return number of threads created.
-   **/
-  virtual unsigned int InitWorkers(unsigned int thread_count) = 0;
-
-  /**
-   * Place an item on the execution queue.  Will be invoked from outside of the calling context.
-   **/
-  virtual void PostTask(std::function<void(void)>& asyncTask) = 0;
-
-  /**
-   * Run the asynchronous tasks associated with this IoService.
-   **/
-  virtual void Run() = 0;
-  /**
-   * Stop running asynchronous tasks associated with this IoService.
-   * All worker threads will return as soon as they finish executing their current task.
-   **/
-  virtual void Stop() = 0;
-};
-
 /**
  * A node exclusion rule provides a simple way of testing if the
  * client should attempt to connect to a node based on the node's

+ 140 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/ioservice.h

@@ -0,0 +1,140 @@
+/**
+ * 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.
+ */
+
+/**
+ * An asio::io_service maintains a queue of asynchronous tasks and invokes them
+ * when they are ready to run.  Async network IO handlers become runnable when
+ * the associated IO operation has completed.  The hdfs::IoService is a thin
+ * wrapper over that object to make it easier to add logging and instrumentation
+ * to tasks that have been queued.
+ *
+ * Lifecycle management:
+ *   -The IoService *shall* outlive any tasks it owns.  Deleting a task
+ *    before it has been run **will** result in dangling reference issues.
+ *   -Dependencies (including transitive dependencies) of pending tasks
+ *    *shall* outlive the task.  Failure to ensure this **will** result in
+ *    danging reference issues.
+ *     -libhdfs++ uses shared_ptr/weak_ptr heavily as a mechanism to ensure
+ *      liveness of dependencies.
+ *     -refcounted pointers in lambda capture lists have a poor track record
+ *      for ensuring liveness in this library; it's easy to omit them because
+ *      the capture list isn't context aware.  Developers are encouraged to
+ *      write callable classes that explicitly list dependencies.
+ *
+ * Constraints on tasks:
+ *   -Tasks and async callbacks *shall* never do blocking IO or sleep().
+ *    At best this hurts performance by preventing worker threads from doing
+ *    useful work.  It may also cause situations that look like deadlocks
+ *    if the worker thread is stalled for long enough.
+ *   -Tasks and async callbacks *shall* not acquire locks that guard resources
+ *    that might be unavailable for an unknown amount of time.  Lock acquisition
+ *    when accessing shared data structures is acceptable and is often required.
+ *   -Tasks and async callbacks *should* not allow exceptions to escape their
+ *    scope since tasks will be executed on a different stack then where they
+ *    were created.  The exception will be caught by the IoService rather than
+ *    being forwarded to the next task.
+ *   -Tasks and async callbacks *should* not rely on thread local storage for
+ *    ancillary context.  The IoService does not support any sort of thread
+ *    affinity that would guarantee tasks Post()ed from one thread will always
+ *    be executed on the same thread.  Applications that only use a single
+ *    worker thread may use TLS but developers should be mindful that throughput
+ *    can no longer be scaled by adding threads.
+ **/
+#ifndef INCLUDE_HDFSPP_IOSERVICE_H_
+#define INCLUDE_HDFSPP_IOSERVICE_H_
+
+#include <memory>
+
+// forward decl
+namespace asio {
+  class io_service;
+}
+
+namespace hdfs {
+
+// (Un)comment this to determine if issues are due to concurrency or logic faults
+// If tests still fail with concurrency disabled it's most likely a logic bug
+#define DISABLE_CONCURRENT_WORKERS
+
+class IoService : public std::enable_shared_from_this<IoService>
+{
+ public:
+  static IoService *New();
+  static std::shared_ptr<IoService> MakeShared();
+  virtual ~IoService();
+
+  /**
+   * Start up as many threads as there are logical processors.
+   * Return number of threads created.
+   **/
+  virtual unsigned int InitDefaultWorkers() = 0;
+
+  /**
+   * Initialize with thread_count handler threads.
+   * If thread count is less than one print a log message and default to one thread.
+   * Return number of threads created.
+   **/
+  virtual unsigned int InitWorkers(unsigned int thread_count) = 0;
+
+  /**
+   * Add a worker thread to existing pool.
+   * Return true on success, false otherwise.
+   **/
+  virtual bool AddWorkerThread() = 0;
+
+  /**
+   * Return the number of worker threads in use.
+   **/
+  virtual unsigned int GetWorkerThreadCount() = 0;
+
+  /**
+   * Enqueue an item for deferred execution.  Non-blocking.
+   * Task will be invoked from outside of the calling context.
+   **/
+  virtual void PostTask(std::function<void(void)> asyncTask) = 0;
+
+  /**
+   * Provide type erasure for lambdas defined inside the argument list.
+   **/
+  template <typename LambdaInstance>
+  inline void PostLambda(LambdaInstance&& func)
+  {
+    std::function<void(void)> typeEraser = func;
+    this->PostTask(func);
+  }
+
+  /**
+   * Run the asynchronous tasks associated with this IoService.
+   **/
+  virtual void Run() = 0;
+  /**
+   * Stop running asynchronous tasks associated with this IoService.
+   * All worker threads will return as soon as they finish executing their current task.
+   **/
+  virtual void Stop() = 0;
+
+  /**
+   * Access underlying io_service object.  Only to be used in asio library calls.
+   * After HDFS-11884 is complete only tests should need direct access to the asio::io_service.
+   **/
+  virtual asio::io_service& GetRaw() = 0;
+};
+
+
+} // namespace hdfs
+#endif // include guard

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc

@@ -17,18 +17,17 @@
  */
 
 #include "hdfspp/hdfspp.h"
+#include "hdfspp/hdfs_ext.h"
 
-#include "fs/filesystem.h"
 #include "common/hdfs_configuration.h"
 #include "common/configuration_loader.h"
 #include "common/logging.h"
+#include "fs/filesystem.h"
+#include "fs/filehandle.h"
 
-#include <hdfs/hdfs.h>
-#include <hdfspp/hdfs_ext.h>
 
 #include <libgen.h>
 #include "limits.h"
-
 #include <string>
 #include <cstring>
 #include <iostream>

+ 1 - 1
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 statinfo.cc fsinfo.cc content_summary.cc locks.cc config_parser.cc)
+add_library(common_obj OBJECT status.cc sasl_digest_md5.cc ioservice_impl.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 locks.cc config_parser.cc)
 add_library(common $<TARGET_OBJECTS:common_obj> $<TARGET_OBJECTS:uriparser2_obj>)
 target_link_libraries(common ${LIB_DL})

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/async_stream.h

@@ -19,12 +19,15 @@
 #ifndef LIB_COMMON_ASYNC_STREAM_H_
 #define LIB_COMMON_ASYNC_STREAM_H_
 
-#include <asio.hpp>
+#include <asio/buffer.hpp>
+#include <asio/error_code.hpp>
+#include <functional>
 
 namespace hdfs {
 
-typedef asio::mutable_buffers_1 MutableBuffers;
-typedef asio::const_buffers_1   ConstBuffers;
+// Contiguous buffer types
+typedef asio::mutable_buffers_1 MutableBuffer;
+typedef asio::const_buffers_1   ConstBuffer;
 
 /*
  * asio-compatible stream implementation.
@@ -35,11 +38,11 @@ typedef asio::const_buffers_1   ConstBuffers;
  */
 class AsyncStream  {
 public:
-  virtual void async_read_some(const MutableBuffers &buf,
+  virtual void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) = 0;
 
-  virtual void async_write_some(const ConstBuffers &buf,
+  virtual void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) = 0;
 };

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h

@@ -20,13 +20,8 @@
 
 #include "continuation.h"
 #include "common/util.h"
-
 #include "hdfspp/status.h"
-
-#include <asio/connect.hpp>
-#include <asio/read.hpp>
 #include <asio/write.hpp>
-#include <asio/ip/tcp.hpp>
 #include <memory>
 
 namespace hdfs {

+ 18 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_ioservice.cc → hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.cc

@@ -16,14 +16,16 @@
  * limitations under the License.
  */
 
-#include "hdfs_ioservice.h"
+#include "ioservice_impl.h"
 
 #include <thread>
 #include <mutex>
 #include <vector>
 
+#include "common/util.h"
 #include "common/logging.h"
 
+
 namespace hdfs {
 
 IoService::~IoService() {}
@@ -99,7 +101,7 @@ void IoServiceImpl::ThreadExitHook() {
   LOG_DEBUG(kAsyncRuntime, << "Worker thread #" << std::this_thread::get_id() << " for IoServiceImpl@" << this << " exiting");
 }
 
-void IoServiceImpl::PostTask(std::function<void(void)>& asyncTask) {
+void IoServiceImpl::PostTask(std::function<void(void)> asyncTask) {
   io_service_.post(asyncTask);
 }
 
@@ -133,14 +135,25 @@ void IoServiceImpl::Run() {
     } catch (const std::exception & e) {
       LOG_WARN(kFileSystem, << "Unexpected exception in libhdfspp worker thread: " << e.what());
     } catch (...) {
-      LOG_WARN(kFileSystem, << "Unexpected value not derived from std::exception in libhdfspp worker thread");
+      LOG_WARN(kFileSystem, << "Caught unexpected value not derived from std::exception in libhdfspp worker thread");
     }
   }
 }
 
-unsigned int IoServiceImpl::get_worker_thread_count() {
+void IoServiceImpl::Stop() {
+  // Note: This doesn't wait for running operations to stop.
+  io_service_.stop();
+}
+
+asio::io_service& IoServiceImpl::GetRaw() {
+  return io_service_;
+}
+
+unsigned int IoServiceImpl::GetWorkerThreadCount() {
   mutex_guard state_lock(state_lock_);
   return worker_threads_.size();
-}
 
 }
+
+
+} // namespace hdfs

+ 12 - 15
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_ioservice.h → hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/ioservice_impl.h

@@ -19,20 +19,16 @@
 #ifndef COMMON_HDFS_IOSERVICE_H_
 #define COMMON_HDFS_IOSERVICE_H_
 
-#include "hdfspp/hdfspp.h"
+#include "hdfspp/ioservice.h"
 
 #include <asio/io_service.hpp>
-#include "common/util.h"
+#include "common/new_delete.h"
 
 #include <mutex>
 #include <thread>
 
 namespace hdfs {
 
-// Uncomment this to determine if issues are due to concurrency or logic faults
-// If tests still fail with concurrency disabled it's most likely a logic bug
-#define DISABLE_CONCURRENT_WORKERS
-
 /*
  *  A thin wrapper over the asio::io_service with a few extras
  *    -manages it's own worker threads
@@ -41,23 +37,24 @@ namespace hdfs {
 
 class IoServiceImpl : public IoService {
  public:
+  MEMCHECKED_CLASS(IoServiceImpl)
   IoServiceImpl() {}
 
-  virtual unsigned int InitDefaultWorkers() override;
-  virtual unsigned int InitWorkers(unsigned int thread_count) override;
-  virtual void PostTask(std::function<void(void)>& asyncTask) override;
-  virtual void Run() override;
-  virtual void Stop() override { io_service_.stop(); }
+  unsigned int InitDefaultWorkers() override;
+  unsigned int InitWorkers(unsigned int thread_count) override;
+  void PostTask(std::function<void(void)> asyncTask) override;
+  void Run() override;
+  void Stop() override;
+  asio::io_service& GetRaw() override;
 
   // Add a single worker thread, in the common case try to avoid this in favor
   // of Init[Default]Workers. Public for use by tests and rare cases where a
   // client wants very explicit control of threading for performance reasons
   // e.g. pinning threads to NUMA nodes.
-  bool AddWorkerThread();
+  bool AddWorkerThread() override;
+
+  unsigned int GetWorkerThreadCount() override;
 
-  // Be very careful about using this: HDFS-10241
-  ::asio::io_service &io_service() { return io_service_; }
-  unsigned int get_worker_thread_count();
  private:
   std::mutex state_lock_;
   ::asio::io_service io_service_;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h

@@ -23,14 +23,11 @@
 
 #include "hdfspp/log.h"
 
-#include <iostream>
 #include <sstream>
 #include <mutex>
 #include <memory>
 #include <thread>
 
-#include <asio/ip/tcp.hpp>
-
 namespace hdfs {
 
 /**

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc

@@ -20,11 +20,12 @@
 
 #include "common/util.h"
 #include "common/logging.h"
+#include "hdfspp/ioservice.h"
 
 #include <sstream>
 #include <utility>
 #include <future>
-#include <memory>
+
 
 namespace hdfs {
 
@@ -35,8 +36,6 @@ ResolvedNamenodeInfo& ResolvedNamenodeInfo::operator=(const NamenodeInfo &info)
   return *this;
 }
 
-
-
 std::string ResolvedNamenodeInfo::str() const {
   std::stringstream ss;
   ss << "ResolvedNamenodeInfo {nameservice: " << nameservice << ", name: " << name << ", uri: " << uri.str();
@@ -58,7 +57,7 @@ std::string ResolvedNamenodeInfo::str() const {
 }
 
 
-bool ResolveInPlace(::asio::io_service *ioservice, ResolvedNamenodeInfo &info) {
+bool ResolveInPlace(std::shared_ptr<IoService> ioservice, ResolvedNamenodeInfo &info) {
   // this isn't very memory friendly, but if it needs to be called often there are bigger issues at hand
   info.endpoints.clear();
   std::vector<ResolvedNamenodeInfo> resolved = BulkResolve(ioservice, {info});
@@ -76,7 +75,7 @@ typedef std::vector<asio::ip::tcp::endpoint> endpoint_vector;
 // RAII wrapper
 class ScopedResolver {
  private:
-  ::asio::io_service *io_service_;
+  std::shared_ptr<IoService> io_service_;
   std::string host_;
   std::string port_;
   ::asio::ip::tcp::resolver::query query_;
@@ -86,8 +85,8 @@ class ScopedResolver {
   // Caller blocks on access if resolution isn't finished
   std::shared_ptr<std::promise<Status>> result_status_;
  public:
-  ScopedResolver(::asio::io_service *service, const std::string &host, const std::string &port) :
-        io_service_(service), host_(host), port_(port), query_(host, port), resolver_(*io_service_)
+  ScopedResolver(std::shared_ptr<IoService> service, const std::string &host, const std::string &port) :
+        io_service_(service), host_(host), port_(port), query_(host, port), resolver_(io_service_->GetRaw())
   {
     if(!io_service_)
       LOG_ERROR(kAsyncRuntime, << "ScopedResolver@" << this << " passed nullptr to io_service");
@@ -140,7 +139,7 @@ class ScopedResolver {
   }
 };
 
-std::vector<ResolvedNamenodeInfo> BulkResolve(::asio::io_service *ioservice, const std::vector<NamenodeInfo> &nodes) {
+std::vector<ResolvedNamenodeInfo> BulkResolve(std::shared_ptr<IoService> ioservice, const std::vector<NamenodeInfo> &nodes) {
   std::vector< std::unique_ptr<ScopedResolver> > resolvers;
   resolvers.reserve(nodes.size());
 

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.h

@@ -20,6 +20,7 @@
 #define COMMON_HDFS_NAMENODE_INFO_H_
 
 #include <asio.hpp>
+
 #include <hdfspp/options.h>
 
 #include <string>
@@ -27,6 +28,9 @@
 
 namespace hdfs {
 
+// Forward decl
+class IoService;
+
 // Internal representation of namenode info that keeps track
 // of its endpoints.
 struct ResolvedNamenodeInfo : public NamenodeInfo {
@@ -38,11 +42,11 @@ struct ResolvedNamenodeInfo : public NamenodeInfo {
 
 // Clear endpoints if set and resolve all of them in parallel.
 // Only successful lookups will be placed in the result set.
-std::vector<ResolvedNamenodeInfo> BulkResolve(::asio::io_service *ioservice, const std::vector<NamenodeInfo> &nodes);
+std::vector<ResolvedNamenodeInfo> BulkResolve(std::shared_ptr<IoService> ioservice, const std::vector<NamenodeInfo> &nodes);
 
 // Clear endpoints, if any, and resolve them again
 // Return true if endpoints were resolved
-bool ResolveInPlace(::asio::io_service *ioservice, ResolvedNamenodeInfo &info);
+bool ResolveInPlace(std::shared_ptr<IoService> ioservice, ResolvedNamenodeInfo &info);
 
 }
 

+ 13 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc

@@ -19,17 +19,25 @@
 #include "common/util.h"
 #include "common/util_c.h"
 
+#include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
 
 #include <exception>
 #include <sstream>
-#include <iostream>
 #include <iomanip>
 #include <thread>
 
 
 namespace hdfs {
 
+Status ToStatus(const ::asio::error_code &ec) {
+  if (ec) {
+    return Status(ec.value(), ec.message().c_str());
+  } else {
+    return Status::OK();
+  }
+}
+
 bool ReadDelimitedPBMessage(::google::protobuf::io::CodedInputStream *in,
                             ::google::protobuf::MessageLite *msg) {
   uint32_t size = 0;
@@ -60,6 +68,10 @@ std::string SerializeDelimitedProtobufMessage(const ::google::protobuf::MessageL
   return buf;
 }
 
+int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg) {
+  size_t size = msg->ByteSize();
+  return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size;
+}
 
 std::string GetRandomClientName() {
   std::vector<unsigned char>buf(8);

+ 9 - 16
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h

@@ -21,16 +21,19 @@
 #include "hdfspp/status.h"
 #include "common/logging.h"
 
-#include <sstream>
 #include <mutex>
 #include <string>
 
 #include <asio/error_code.hpp>
 #include <openssl/rand.h>
-
-#include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/coded_stream.h>
-#include <asio.hpp>
+
+
+namespace google {
+  namespace protobuf {
+    class MessageLite;
+  }
+}
 
 namespace hdfs {
 
@@ -38,20 +41,11 @@ namespace hdfs {
 typedef std::lock_guard<std::mutex> mutex_guard;
 
 
-static inline Status ToStatus(const ::asio::error_code &ec) {
-  if (ec) {
-    return Status(ec.value(), ec.message().c_str());
-  } else {
-    return Status::OK();
-  }
-}
+Status ToStatus(const ::asio::error_code &ec);
 
 // Determine size of buffer that needs to be allocated in order to serialize msg
 // in delimited format
-static inline int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg) {
-  size_t size = msg->ByteSize();
-  return ::google::protobuf::io::CodedOutputStream::VarintSize32(size) + size;
-}
+int DelimitedPBMessageSize(const ::google::protobuf::MessageLite *msg);
 
 // Construct msg from the input held in the CodedInputStream
 // return false on failure, otherwise return true
@@ -84,7 +78,6 @@ bool lock_held(T & mutex) {
 std::string SafeDisconnect(asio::ip::tcp::socket *sock);
 
 
-
 // The following helper function is used for classes that look like the following:
 //
 // template <typename socket_like_object>

+ 22 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.cc

@@ -24,14 +24,14 @@ namespace hdfs {
 DataNodeConnection::~DataNodeConnection(){}
 DataNodeConnectionImpl::~DataNodeConnectionImpl(){}
 
-DataNodeConnectionImpl::DataNodeConnectionImpl(asio::io_service * io_service,
-                                                const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
-                                                const hadoop::common::TokenProto *token,
-                                                LibhdfsEvents *event_handlers) : event_handlers_(event_handlers)
+DataNodeConnectionImpl::DataNodeConnectionImpl(std::shared_ptr<IoService> io_service,
+                                               const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
+                                               const hadoop::common::TokenProto *token,
+                                               LibhdfsEvents *event_handlers) : event_handlers_(event_handlers)
 {
   using namespace ::asio::ip;
 
-  conn_.reset(new tcp::socket(*io_service));
+  conn_.reset(new tcp::socket(io_service->GetRaw()));
   auto datanode_addr = dn_proto.id();
   endpoints_[0] = tcp::endpoint(address::from_string(datanode_addr.ipaddr()),
                                   datanode_addr.xferport());
@@ -68,5 +68,22 @@ void DataNodeConnectionImpl::Cancel() {
   }
 }
 
+void DataNodeConnectionImpl::async_read_some(const MutableBuffer &buf,
+             std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
+{
+  event_handlers_->call("DN_read_req", "", "", buf.end() - buf.begin());
+
+  mutex_guard state_lock(state_lock_);
+  conn_->async_read_some(buf, handler);
+}
+
+void DataNodeConnectionImpl::async_write_some(const ConstBuffer &buf,
+             std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
+{
+  event_handlers_->call("DN_write_req", "", "", buf.end() - buf.begin());
+
+  mutex_guard state_lock(state_lock_);
+  conn_->async_write_some(buf, handler);
+}
 
 }

+ 7 - 19
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h

@@ -18,7 +18,7 @@
 #ifndef LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 #define LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
 #include "ClientNamenodeProtocol.pb.h"
 #include "common/libhdfs_events_impl.h"
@@ -58,13 +58,14 @@ private:
   // held (briefly) while posting async ops to the asio task queue
   std::mutex state_lock_;
 public:
+  MEMCHECKED_CLASS(DataNodeConnectionImpl)
   std::unique_ptr<asio::ip::tcp::socket, SocketDeleter> conn_;
   std::array<asio::ip::tcp::endpoint, 1> endpoints_;
   std::string uuid_;
   LibhdfsEvents *event_handlers_;
 
   virtual ~DataNodeConnectionImpl();
-  DataNodeConnectionImpl(asio::io_service * io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
+  DataNodeConnectionImpl(std::shared_ptr<IoService> io_service, const ::hadoop::hdfs::DatanodeInfoProto &dn_proto,
                           const hadoop::common::TokenProto *token,
                           LibhdfsEvents *event_handlers);
 
@@ -72,24 +73,11 @@ public:
 
   void Cancel() override;
 
-  void async_read_some(const MutableBuffers &buf,
-                         std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
-                       override {
-    event_handlers_->call("DN_read_req", "", "", buf.end() - buf.begin());
+  void async_read_some(const MutableBuffer &buf,
+                       std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler) override;
 
-
-    mutex_guard state_lock(state_lock_);
-    conn_->async_read_some(buf, handler);
-  };
-
-  void async_write_some(const ConstBuffers &buf,
-                          std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler)
-                        override {
-    event_handlers_->call("DN_write_req", "", "", buf.end() - buf.begin());
-
-    mutex_guard state_lock(state_lock_);
-    conn_->async_write_some(buf, handler);
-  }
+  void async_write_some(const ConstBuffer &buf,
+                        std::function<void (const asio::error_code & error, std::size_t bytes_transferred) > handler) override;
 };
 
 }

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc

@@ -36,10 +36,10 @@ FileHandle::~FileHandle() {}
 
 FileHandleImpl::FileHandleImpl(const std::string & cluster_name,
                                const std::string & path,
-                               ::asio::io_service *io_service, const std::string &client_name,
-                                 const std::shared_ptr<const struct FileInfo> file_info,
-                                 std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
-                                 std::shared_ptr<LibhdfsEvents> event_handlers)
+                               std::shared_ptr<IoService> io_service, const std::string &client_name,
+                               const std::shared_ptr<const struct FileInfo> file_info,
+                               std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
+                               std::shared_ptr<LibhdfsEvents> event_handlers)
     : cluster_name_(cluster_name), path_(path), io_service_(io_service), client_name_(client_name), file_info_(file_info),
       bad_node_tracker_(bad_data_nodes), offset_(0), cancel_state_(CancelTracker::New()), event_handlers_(event_handlers), bytes_read_(0) {
   LOG_TRACE(kFileHandle, << "FileHandleImpl::FileHandleImpl("
@@ -167,7 +167,7 @@ bool FileHandleImpl::CheckSeekBounds(ssize_t desired_position) {
  * on the FileHandle
  */
 void FileHandleImpl::AsyncPreadSome(
-    size_t offset, const MutableBuffers &buffers,
+    size_t offset, const MutableBuffer &buffer,
     std::shared_ptr<NodeExclusionRule> excluded_nodes,
     const std::function<void(const Status &, const std::string &, size_t)> handler) {
   using ::hadoop::hdfs::DatanodeInfoProto;
@@ -233,7 +233,7 @@ void FileHandleImpl::AsyncPreadSome(
 
   uint64_t offset_within_block = offset - block->offset();
   uint64_t size_within_block = std::min<uint64_t>(
-      block->b().numbytes() - offset_within_block, asio::buffer_size(buffers));
+      block->b().numbytes() - offset_within_block, asio::buffer_size(buffer));
 
   LOG_DEBUG(kFileHandle, << "FileHandleImpl::AsyncPreadSome("
             << FMT_THIS_ADDR << "), ...) Datanode hostname=" << dnHostName << ", IP Address=" << dnIpAddr
@@ -268,7 +268,7 @@ void FileHandleImpl::AsyncPreadSome(
     handler(status, dn_id, transferred);
   };
 
-  auto connect_handler = [handler,event_handlers,cluster_name,path,read_handler,block,offset_within_block,size_within_block, buffers, reader, dn_id, client_name]
+  auto connect_handler = [handler,event_handlers,cluster_name,path,read_handler,block,offset_within_block,size_within_block, buffer, reader, dn_id, client_name]
           (Status status, std::shared_ptr<DataNodeConnection> dn) {
     (void)dn;
     event_response event_resp = event_handlers->call(FILE_DN_CONNECT_EVENT, cluster_name.c_str(), path.c_str(), 0);
@@ -281,7 +281,7 @@ void FileHandleImpl::AsyncPreadSome(
     if (status.ok()) {
       reader->AsyncReadBlock(
           client_name, *block, offset_within_block,
-          asio::buffer(buffers, size_within_block), read_handler);
+          asio::buffer(buffer, size_within_block), read_handler);
     } else {
       handler(status, dn_id, 0);
     }
@@ -307,7 +307,7 @@ std::shared_ptr<BlockReader> FileHandleImpl::CreateBlockReader(const BlockReader
 }
 
 std::shared_ptr<DataNodeConnection> FileHandleImpl::CreateDataNodeConnection(
-    ::asio::io_service * io_service,
+    std::shared_ptr<IoService> io_service,
     const ::hadoop::hdfs::DatanodeInfoProto & dn,
     const hadoop::common::TokenProto * token) {
   LOG_TRACE(kFileHandle, << "FileHandleImpl::CreateDataNodeConnection("

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h

@@ -18,7 +18,7 @@
 #ifndef LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 #define LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
 #include "common/libhdfs_events_impl.h"
@@ -26,12 +26,10 @@
 #include "reader/fileinfo.h"
 #include "reader/readergroup.h"
 
-#include "asio.hpp"
 #include "bad_datanode_tracker.h"
 #include "ClientNamenodeProtocol.pb.h"
 
 #include <mutex>
-#include <iostream>
 
 namespace hdfs {
 
@@ -53,7 +51,7 @@ public:
   MEMCHECKED_CLASS(FileHandleImpl)
   FileHandleImpl(const std::string & cluster_name,
                  const std::string & path,
-                 ::asio::io_service *io_service, const std::string &client_name,
+                 std::shared_ptr<IoService> io_service, const std::string &client_name,
                   const std::shared_ptr<const struct FileInfo> file_info,
                   std::shared_ptr<BadDataNodeTracker> bad_data_nodes,
                   std::shared_ptr<LibhdfsEvents> event_handlers);
@@ -93,7 +91,7 @@ public:
    * If trying to begin a read past the EOF, status will be Status::InvalidOffset.
    *
    */
-  void AsyncPreadSome(size_t offset, const MutableBuffers &buffers,
+  void AsyncPreadSome(size_t offset, const MutableBuffer &buffer,
                       std::shared_ptr<NodeExclusionRule> excluded_nodes,
                       const std::function<void(const Status &status,
                       const std::string &dn_id, size_t bytes_read)> handler);
@@ -124,13 +122,13 @@ protected:
                                                          std::shared_ptr<DataNodeConnection> dn,
                                                          std::shared_ptr<hdfs::LibhdfsEvents> event_handlers);
   virtual std::shared_ptr<DataNodeConnection> CreateDataNodeConnection(
-      ::asio::io_service *io_service,
+      std::shared_ptr<IoService> io_service,
       const ::hadoop::hdfs::DatanodeInfoProto & dn,
       const hadoop::common::TokenProto * token);
 private:
   const std::string cluster_name_;
   const std::string path_;
-  ::asio::io_service * const io_service_;
+  std::shared_ptr<IoService> io_service_;
   const std::string client_name_;
   const std::shared_ptr<const struct FileInfo> file_info_;
   std::shared_ptr<BadDataNodeTracker> bad_node_tracker_;

+ 58 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc

@@ -18,6 +18,7 @@
 
 #include "filesystem.h"
 
+#include "filehandle.h"
 #include "common/namenode_info.h"
 
 #include <functional>
@@ -104,6 +105,54 @@ FileSystem *FileSystem::New() {
  *                    FILESYSTEM IMPLEMENTATION
  ****************************************************************************/
 
+struct FileSystemImpl::FindSharedState {
+  //Name pattern (can have wild-cards) to find
+  const std::string name;
+  //Maximum depth to recurse after the end of path is reached.
+  //Can be set to 0 for pure path globbing and ignoring name pattern entirely.
+  const uint32_t maxdepth;
+  //Vector of all sub-directories from the path argument (each can have wild-cards)
+  std::vector<std::string> dirs;
+  //Callback from Find
+  const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler;
+  //outstanding_requests is incremented once for every GetListing call.
+  std::atomic<uint64_t> outstanding_requests;
+  //Boolean needed to abort all recursion on error or on user command
+  std::atomic<bool> aborted;
+  //Shared variables will need protection with a lock
+  std::mutex lock;
+  FindSharedState(const std::string path_, const std::string name_, const uint32_t maxdepth_,
+              const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler_,
+              uint64_t outstanding_recuests_, bool aborted_)
+      : name(name_),
+        maxdepth(maxdepth_),
+        handler(handler_),
+        outstanding_requests(outstanding_recuests_),
+        aborted(aborted_),
+        lock() {
+    //Constructing the list of sub-directories
+    std::stringstream ss(path_);
+    if(path_.back() != '/'){
+      ss << "/";
+    }
+    for (std::string token; std::getline(ss, token, '/'); ) {
+      dirs.push_back(token);
+    }
+  }
+};
+
+struct FileSystemImpl::FindOperationalState {
+  const std::string path;
+  const uint32_t depth;
+  const bool search_path;
+  FindOperationalState(const std::string path_, const uint32_t depth_, const bool search_path_)
+      : path(path_),
+        depth(depth_),
+        search_path(search_path_) {
+  }
+};
+
+
 const std::string get_effective_user_name(const std::string &user_name) {
   if (!user_name.empty())
     return user_name;
@@ -134,10 +183,10 @@ const std::string get_effective_user_name(const std::string &user_name) {
 }
 
 FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_name, const Options &options) :
-     io_service_(static_cast<IoServiceImpl *>(io_service)), options_(options),
+     io_service_(io_service), options_(options),
      client_name_(GetRandomClientName()),
      nn_(
-       &io_service_->io_service(), options, client_name_,
+       io_service_, options, client_name_,
        get_effective_user_name(user_name), kNamenodeProtocol,
        kNamenodeProtocolVersion
      ),
@@ -166,10 +215,10 @@ FileSystemImpl::FileSystemImpl(IoService *&io_service, const std::string &user_n
 }
 
 FileSystemImpl::FileSystemImpl(std::shared_ptr<IoService> io_service, const std::string& user_name, const Options &options) :
-     io_service_(std::static_pointer_cast<IoServiceImpl>(io_service)), options_(options),
+     io_service_(io_service), options_(options),
      client_name_(GetRandomClientName()),
      nn_(
-       &io_service_->io_service(), options, client_name_,
+       io_service_, options, client_name_,
        get_effective_user_name(user_name), kNamenodeProtocol,
        kNamenodeProtocolVersion
      ),
@@ -178,7 +227,7 @@ FileSystemImpl::FileSystemImpl(std::shared_ptr<IoService> io_service, const std:
 {
   LOG_DEBUG(kFileSystem, << "FileSystemImpl::FileSystemImpl("
                          << FMT_THIS_ADDR << ", shared IoService@" << io_service_.get() << ") called");
-  int worker_thread_count = io_service_->get_worker_thread_count();
+  int worker_thread_count = io_service_->GetWorkerThreadCount();
   if(worker_thread_count < 1) {
     LOG_WARN(kFileSystem, << "FileSystemImpl::FileSystemImpl IoService provided doesn't have any worker threads. "
                           << "It needs at least 1 worker to connect to an HDFS cluster.")
@@ -217,7 +266,7 @@ void FileSystemImpl::Connect(const std::string &server,
   auto name_service = options_.services.find(server);
   if(name_service != options_.services.end()) {
     cluster_name_ = name_service->first;
-    resolved_namenodes = BulkResolve(&io_service_->io_service(), name_service->second);
+    resolved_namenodes = BulkResolve(io_service_, name_service->second);
   } else {
     cluster_name_ = server + ":" + service;
 
@@ -230,7 +279,7 @@ void FileSystemImpl::Connect(const std::string &server,
       handler(Status::Error(("Invalid namenode " + cluster_name_ + " in config").c_str()), this);
     }
 
-    resolved_namenodes = BulkResolve(&io_service_->io_service(), {tmp_info});
+    resolved_namenodes = BulkResolve(io_service_, {tmp_info});
   }
 
   for(unsigned int i=0;i<resolved_namenodes.size();i++) {
@@ -282,7 +331,7 @@ int FileSystemImpl::WorkerThreadCount() {
   if(!io_service_) {
     return -1;
   } else {
-    return io_service_->get_worker_thread_count();
+    return io_service_->GetWorkerThreadCount();
   }
 }
 
@@ -339,7 +388,7 @@ void FileSystemImpl::Open(
         LOG_DEBUG(kFileSystem, << "Operation not allowed on standby datanode");
       }
     }
-    handler(stat, stat.ok() ? new FileHandleImpl(cluster_name_, path, &io_service_->io_service(), client_name_, file_info, bad_node_tracker_, event_handlers_)
+    handler(stat, stat.ok() ? new FileHandleImpl(cluster_name_, path, io_service_, client_name_, file_info, bad_node_tracker_, event_handlers_)
                             : nullptr);
   });
 }

+ 12 - 54
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h

@@ -18,19 +18,18 @@
 #ifndef LIBHDFSPP_LIB_FS_FILESYSTEM_H_
 #define LIBHDFSPP_LIB_FS_FILESYSTEM_H_
 
-#include "filehandle.h"
-#include "hdfspp/hdfspp.h"
+#include "namenode_operations.h"
 #include "fs/bad_datanode_tracker.h"
-#include "reader/block_reader.h"
+#include "hdfspp/hdfspp.h"
 #include "reader/fileinfo.h"
 
-#include "asio.hpp"
-
 #include <thread>
-#include "namenode_operations.h"
 
 namespace hdfs {
 
+class FileHandle;
+
+
 /*
  * FileSystem: The consumer's main point of interaction with the cluster as
  * a whole.
@@ -48,6 +47,7 @@ public:
   MEMCHECKED_CLASS(FileSystemImpl)
   typedef std::function<void(const Status &, FileSystem *)> ConnectCallback;
 
+  // Note: Longer term it'd be cleaner to take a rvalue reference to a shared_ptr to get ownership
   explicit FileSystemImpl(IoService *&io_service, const std::string& user_name, const Options &options);
   explicit FileSystemImpl(std::shared_ptr<IoService>, const std::string& user_name, const Options &options);
   ~FileSystemImpl() override;
@@ -215,7 +215,7 @@ private:
    *  A side effect of this is that requests may outlive the RpcEngine they
    *  reference.
    **/
-  std::shared_ptr<IoServiceImpl> io_service_;
+  std::shared_ptr<IoService> io_service_;
   const Options options_;
   const std::string client_name_;
   std::string cluster_name_;
@@ -234,53 +234,11 @@ private:
 
   void GetListingShim(const Status &stat, const std::vector<StatInfo> & stat_infos, bool has_more,
               std::string path, const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> &handler);
-
-  struct FindSharedState {
-    //Name pattern (can have wild-cards) to find
-    const std::string name;
-    //Maximum depth to recurse after the end of path is reached.
-    //Can be set to 0 for pure path globbing and ignoring name pattern entirely.
-    const uint32_t maxdepth;
-    //Vector of all sub-directories from the path argument (each can have wild-cards)
-    std::vector<std::string> dirs;
-    //Callback from Find
-    const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler;
-    //outstanding_requests is incremented once for every GetListing call.
-    std::atomic<uint64_t> outstanding_requests;
-    //Boolean needed to abort all recursion on error or on user command
-    std::atomic<bool> aborted;
-    //Shared variables will need protection with a lock
-    std::mutex lock;
-    FindSharedState(const std::string path_, const std::string name_, const uint32_t maxdepth_,
-                const std::function<bool(const Status &, const std::vector<StatInfo> &, bool)> handler_,
-                uint64_t outstanding_recuests_, bool aborted_)
-        : name(name_),
-          maxdepth(maxdepth_),
-          handler(handler_),
-          outstanding_requests(outstanding_recuests_),
-          aborted(aborted_),
-          lock() {
-      //Constructing the list of sub-directories
-      std::stringstream ss(path_);
-      if(path_.back() != '/'){
-        ss << "/";
-      }
-      for (std::string token; std::getline(ss, token, '/'); ) {
-        dirs.push_back(token);
-      }
-    }
-  };
-
-  struct FindOperationalState {
-    const std::string path;
-    const uint32_t depth;
-    const bool search_path;
-    FindOperationalState(const std::string path_, const uint32_t depth_, const bool search_path_)
-        : path(path_),
-          depth(depth_),
-          search_path(search_path_) {
-    }
-  };
+  /**
+   * Helper struct to store state for recursive find
+   */
+  struct FindSharedState;
+  struct FindOperationalState;
 
   void FindShim(const Status &stat, const std::vector<StatInfo> & stat_infos,
                 bool directory_has_more, std::shared_ptr<FindOperationalState> current_state, std::shared_ptr<FindSharedState> shared_state);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h

@@ -42,7 +42,7 @@ namespace hdfs {
 class NameNodeOperations {
 public:
   MEMCHECKED_CLASS(NameNodeOperations)
-  NameNodeOperations(::asio::io_service *io_service, const Options &options,
+  NameNodeOperations(std::shared_ptr<IoService> io_service, const Options &options,
             const std::string &client_name, const std::string &user_name,
             const char *protocol_name, int protocol_version) :
   io_service_(io_service),
@@ -119,7 +119,7 @@ private:
   static void DirectoryListingProtoToStatInfo(std::shared_ptr<std::vector<StatInfo>> stat_infos, const ::hadoop::hdfs::DirectoryListingProto & dl);
   static void GetFsStatsResponseProtoToFsInfo(hdfs::FsInfo & fs_info, const std::shared_ptr<::hadoop::hdfs::GetFsStatsResponseProto> & fs);
 
-  ::asio::io_service * io_service_;
+  std::shared_ptr<IoService> io_service_;
 
   // This is the only permanent owner of the RpcEngine, however the RPC layer
   // needs to reference count it prevent races during FileSystem destruction.

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc

@@ -431,7 +431,7 @@ private:
   std::shared_ptr<DataNodeConnection> shared_conn_;
 };
 
-void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
+void BlockReaderImpl::AsyncReadPacket(const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler)
 {
   assert(state_ != kOpen && "Not connected");
@@ -450,7 +450,7 @@ void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
       .Push(new ReadChecksum(this))
       .Push(new ReadPadding(this))
       .Push(new ReadData(
-          this, m->state().bytes_transferred, buffers))
+          this, m->state().bytes_transferred, buffer))
       .Push(new AckRead(this));
 
   auto self = this->shared_from_this();
@@ -460,14 +460,14 @@ void BlockReaderImpl::AsyncReadPacket(const MutableBuffers &buffers,
 }
 
 
-size_t BlockReaderImpl::ReadPacket(const MutableBuffers &buffers, Status *status)
+size_t BlockReaderImpl::ReadPacket(const MutableBuffer &buffer, Status *status)
 {
   LOG_TRACE(kBlockReader, << "BlockReaderImpl::ReadPacket called");
 
   size_t transferred = 0;
   auto done = std::make_shared<std::promise<void>>();
   auto future = done->get_future();
-  AsyncReadPacket(buffers,
+  AsyncReadPacket(buffer,
                   [status, &transferred, done](const Status &stat, size_t t) {
                     *status = stat;
                     transferred = t;
@@ -504,7 +504,7 @@ private:
 
 struct BlockReaderImpl::ReadBlockContinuation : continuation::Continuation
 {
-  ReadBlockContinuation(BlockReader *reader, MutableBuffers buffer, size_t *transferred)
+  ReadBlockContinuation(BlockReader *reader, MutableBuffer buffer, size_t *transferred)
       : reader_(reader), buffer_(buffer), buffer_size_(asio::buffer_size(buffer)), transferred_(transferred) {}
 
   virtual void Run(const Next &next) override {
@@ -517,7 +517,7 @@ struct BlockReaderImpl::ReadBlockContinuation : continuation::Continuation
 
 private:
   BlockReader *reader_;
-  const MutableBuffers buffer_;
+  const MutableBuffer buffer_;
   const size_t buffer_size_;
   size_t *transferred_;
   std::function<void(const Status &)> next_;
@@ -542,7 +542,7 @@ void BlockReaderImpl::AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block,
     size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler)
 {
   LOG_TRACE(kBlockReader, << "BlockReaderImpl::AsyncReadBlock("
@@ -551,10 +551,10 @@ void BlockReaderImpl::AsyncReadBlock(
   auto m = continuation::Pipeline<size_t>::Create(cancel_state_);
   size_t * bytesTransferred = &m->state();
 
-  size_t size = asio::buffer_size(buffers);
+  size_t size = asio::buffer_size(buffer);
 
   m->Push(new RequestBlockContinuation(this, client_name, &block.b(), size, offset))
-    .Push(new ReadBlockContinuation(this, buffers, bytesTransferred));
+    .Push(new ReadBlockContinuation(this, buffer, bytesTransferred));
 
   m->Run([handler] (const Status &status, const size_t totalBytesTransferred) {
     handler(status, totalBytesTransferred);

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h

@@ -72,11 +72,11 @@ public:
   virtual void AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block, size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler) = 0;
 
   virtual void AsyncReadPacket(
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler) = 0;
 
   virtual void AsyncRequestBlock(
@@ -98,7 +98,7 @@ public:
         chunk_padding_bytes_(0), cancel_state_(cancel_state), event_handlers_(event_handlers.get()) {}
 
   virtual void AsyncReadPacket(
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t bytes_transferred)> &handler) override;
 
   virtual void AsyncRequestBlock(
@@ -111,12 +111,12 @@ public:
   virtual void AsyncReadBlock(
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block, size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler) override;
 
   virtual void CancelOperation() override;
 
-  size_t ReadPacket(const MutableBuffers &buffers, Status *status);
+  size_t ReadPacket(const MutableBuffer &buffer, Status *status);
 
   Status RequestBlock(
     const std::string &client_name,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h

@@ -44,13 +44,13 @@ public:
 
   template <class Handler> void Handshake(const Handler &next);
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     stream_->async_read_some(buf, handler);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     stream_->async_write_some(buf, handler);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.cc

@@ -35,7 +35,7 @@ static std::string format_endpoints(const std::vector<::asio::ip::tcp::endpoint>
 }
 
 HANamenodeTracker::HANamenodeTracker(const std::vector<ResolvedNamenodeInfo> &servers,
-                                     ::asio::io_service *ioservice,
+                                     std::shared_ptr<IoService> ioservice,
                                      std::shared_ptr<LibhdfsEvents> event_handlers)
                   : enabled_(false), resolved_(false),
                     ioservice_(ioservice), event_handlers_(event_handlers)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/namenode_tracker.h

@@ -40,7 +40,7 @@ namespace hdfs {
 class HANamenodeTracker {
  public:
   HANamenodeTracker(const std::vector<ResolvedNamenodeInfo> &servers,
-                    ::asio::io_service *ioservice,
+                    std::shared_ptr<IoService> ioservice,
                     std::shared_ptr<LibhdfsEvents> event_handlers_);
 
   virtual ~HANamenodeTracker();
@@ -66,7 +66,7 @@ class HANamenodeTracker {
   bool resolved_;
 
   // Keep service in case a second round of DNS lookup is required
-  ::asio::io_service *ioservice_;
+  std::shared_ptr<IoService> ioservice_;
 
   // Event handlers, for now this is the simplest place to catch all failover events
   // and push info out to client application.  Possibly move into RPCEngine.

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc

@@ -20,6 +20,7 @@
 #include "request.h"
 #include "rpc_engine.h"
 #include "sasl_protocol.h"
+#include "hdfspp/ioservice.h"
 
 #include "RpcHeader.pb.h"
 #include "ProtobufRpcEngine.pb.h"
@@ -118,7 +119,7 @@ Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, const std::string &m
     : engine_(engine),
       method_name_(method_name),
       call_id_(call_id),
-      timer_(engine->io_service()),
+      timer_(engine->io_service()->GetRaw()),
       handler_(std::move(handler)),
       retry_count_(engine->retry_policy() ? 0 : kNoRetry),
       failover_count_(0)
@@ -129,7 +130,7 @@ Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, const std::string &m
 Request::Request(std::shared_ptr<LockFreeRpcEngine> engine, Handler &&handler)
     : engine_(engine),
       call_id_(-1/*Handshake ID*/),
-      timer_(engine->io_service()),
+      timer_(engine->io_service()->GetRaw()),
       handler_(std::move(handler)),
       retry_count_(engine->retry_policy() ? 0 : kNoRetry),
       failover_count_(0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h

@@ -83,7 +83,7 @@ class RpcConnection : public std::enable_shared_from_this<RpcConnection> {
   void SetAuthInfo(const AuthInfo& auth_info);
 
   std::weak_ptr<LockFreeRpcEngine> engine() { return engine_; }
-  ::asio::io_service *GetIoService();
+  std::shared_ptr<IoService> GetIoService();
 
  protected:
   struct Response {

+ 19 - 13
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc

@@ -70,27 +70,27 @@ RpcConnection::RpcConnection(std::shared_ptr<LockFreeRpcEngine> engine)
     : engine_(engine),
       connected_(kNotYetConnected) {}
 
-::asio::io_service *RpcConnection::GetIoService() {
+std::shared_ptr<IoService> RpcConnection::GetIoService() {
   std::shared_ptr<LockFreeRpcEngine> pinnedEngine = engine_.lock();
   if(!pinnedEngine) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid RpcEngine");
     return nullptr;
   }
 
-  return &pinnedEngine->io_service();
+  return pinnedEngine->io_service();
 }
 
 void RpcConnection::StartReading() {
   auto shared_this = shared_from_this();
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return;
   }
 
-  service->post([shared_this, this] () {
-    OnRecvCompleted(::asio::error_code(), 0);
-  });
+  service->PostLambda(
+    [shared_this, this] () { OnRecvCompleted(::asio::error_code(), 0); }
+  );
 }
 
 void RpcConnection::HandshakeComplete(const Status &s) {
@@ -164,13 +164,14 @@ void RpcConnection::ContextComplete(const Status &s) {
 void RpcConnection::AsyncFlushPendingRequests() {
   std::shared_ptr<RpcConnection> shared_this = shared_from_this();
 
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return;
   }
 
-  service->post([shared_this, this]() {
+  std::function<void()> task = [shared_this, this]()
+  {
     std::lock_guard<std::mutex> state_lock(connection_state_lock_);
 
     LOG_TRACE(kRPC, << "RpcConnection::AsyncFlushPendingRequests called (connected=" << ToString(connected_) << ")");
@@ -178,7 +179,10 @@ void RpcConnection::AsyncFlushPendingRequests() {
     if (!outgoing_request_) {
       FlushPendingRequests();
     }
-  });
+  };
+
+  service->PostTask(task);
+
 }
 
 Status RpcConnection::HandleRpcResponse(std::shared_ptr<Response> response) {
@@ -228,15 +232,17 @@ Status RpcConnection::HandleRpcResponse(std::shared_ptr<Response> response) {
     return status;
   }
 
-  ::asio::io_service *service = GetIoService();
+  std::shared_ptr<IoService> service = GetIoService();
   if(!service) {
     LOG_ERROR(kRPC, << "RpcConnection@" << this << " attempted to access invalid IoService");
     return Status::Error("RpcConnection attempted to access invalid IoService");
   }
 
-  service->post([req, response, status]() {
-    req->OnResponseArrived(response->in.get(), status);  // Never call back while holding a lock
-  });
+  service->PostLambda(
+    [req, response, status]() {
+      req->OnResponseArrived(response->in.get(), status);  // Never call back while holding a lock
+    }
+  );
 
   return Status::OK();
 }

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h

@@ -26,6 +26,7 @@
 #include "common/logging.h"
 #include "common/util.h"
 #include "common/libhdfs_events_impl.h"
+#include "hdfspp/ioservice.h"
 
 #include <asio/connect.hpp>
 #include <asio/read.hpp>
@@ -76,8 +77,8 @@ template <class Socket>
 RpcConnectionImpl<Socket>::RpcConnectionImpl(std::shared_ptr<RpcEngine> engine)
     : RpcConnection(engine),
       options_(engine->options()),
-      socket_(engine->io_service()),
-      connect_timer_(engine->io_service())
+      socket_(engine->io_service()->GetRaw()),
+      connect_timer_(engine->io_service()->GetRaw())
 {
       LOG_TRACE(kRPC, << "RpcConnectionImpl::RpcConnectionImpl called &" << (void*)this);
 }
@@ -353,7 +354,7 @@ void RpcConnectionImpl<Socket>::FlushPendingRequests() {
                         OnSendCompleted(ec, size);
                       });
   } else {  // Nothing to send for this request, inform the handler immediately
-    ::asio::io_service *service = GetIoService();
+    std::shared_ptr<IoService> service = GetIoService();
     if(!service) {
       LOG_ERROR(kRPC, << "RpcConnectionImpl@" << this << " attempted to access null IoService");
       // No easy way to bail out of this context, but the only way to get here is when
@@ -361,7 +362,7 @@ void RpcConnectionImpl<Socket>::FlushPendingRequests() {
       return;
     }
 
-    service->post(
+    service->PostTask(
         // Never hold locks when calling a callback
         [req]() { req->OnResponseArrived(nullptr, Status::OK()); }
     );

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc

@@ -30,7 +30,7 @@ template <class T>
 using optional = std::experimental::optional<T>;
 
 
-RpcEngine::RpcEngine(::asio::io_service *io_service, const Options &options,
+RpcEngine::RpcEngine(std::shared_ptr<IoService> io_service, const Options &options,
                      const std::string &client_name, const std::string &user_name,
                      const char *protocol_name, int protocol_version)
     : io_service_(io_service),
@@ -40,7 +40,7 @@ RpcEngine::RpcEngine(::asio::io_service *io_service, const Options &options,
       protocol_name_(protocol_name),
       protocol_version_(protocol_version),
       call_id_(0),
-      retry_timer(*io_service),
+      retry_timer(io_service->GetRaw()),
       event_handlers_(std::make_shared<LibhdfsEvents>()),
       connect_canceled_(false)
 {
@@ -86,7 +86,7 @@ bool RpcEngine::CancelPendingConnect() {
 
 void RpcEngine::Shutdown() {
   LOG_DEBUG(kRPC, << "RpcEngine::Shutdown called");
-  io_service_->post([this]() {
+  io_service_->PostLambda([this]() {
     std::lock_guard<std::mutex> state_lock(engine_state_lock_);
     conn_.reset();
   });
@@ -154,7 +154,7 @@ void RpcEngine::AsyncRpc(
 
   // In case user-side code isn't checking the status of Connect before doing RPC
   if(connect_canceled_) {
-    io_service_->post(
+    io_service_->PostLambda(
         [handler](){ handler(Status::Canceled()); }
     );
     return;
@@ -190,7 +190,7 @@ void RpcEngine::AsyncRpcCommsError(
     std::vector<std::shared_ptr<Request>> pendingRequests) {
   LOG_ERROR(kRPC, << "RpcEngine::AsyncRpcCommsError called; status=\"" << status.ToString() << "\" conn=" << failedConnection.get() << " reqs=" << std::to_string(pendingRequests.size()));
 
-  io_service().post([this, status, failedConnection, pendingRequests]() {
+  io_service_->PostLambda([this, status, failedConnection, pendingRequests]() {
     RpcCommsError(status, failedConnection, pendingRequests);
   });
 }
@@ -238,7 +238,7 @@ void RpcEngine::RpcCommsError(
       //    on.  There might be a good argument for caching the first error
       //    rather than the last one, that gets messy
 
-      io_service().post([req, status]() {
+      io_service()->PostLambda([req, status]() {
         req->OnResponseArrived(nullptr, status);  // Never call back while holding a lock
       });
       it = pendingRequests.erase(it);
@@ -283,7 +283,7 @@ void RpcEngine::RpcCommsError(
 
           for(unsigned int i=0; i<pendingRequests.size(); i++) {
             std::shared_ptr<Request> sharedCurrentRequest = pendingRequests[i];
-            io_service().post([sharedCurrentRequest, badEndpointStatus]() {
+            io_service()->PostLambda([sharedCurrentRequest, badEndpointStatus]() {
               sharedCurrentRequest->OnResponseArrived(nullptr, badEndpointStatus);  // Never call back while holding a lock
             });
           }

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h

@@ -60,6 +60,7 @@ class RpcConnection;
 class SaslProtocol;
 class RpcConnection;
 class Request;
+class IoService;
 
 /*
  * These methods of the RpcEngine will never acquire locks, and are safe for
@@ -83,7 +84,7 @@ public:
   virtual const std::string &user_name() = 0;
   virtual const std::string &protocol_name() = 0;
   virtual int protocol_version() = 0;
-  virtual ::asio::io_service &io_service() = 0;
+  virtual std::shared_ptr<IoService> io_service() const = 0;
   virtual const Options &options() = 0;
 };
 
@@ -107,7 +108,7 @@ class RpcEngine : public LockFreeRpcEngine, public std::enable_shared_from_this<
     kCallIdSasl = -33
   };
 
-  RpcEngine(::asio::io_service *io_service, const Options &options,
+  RpcEngine(std::shared_ptr<IoService> service, const Options &options,
             const std::string &client_name, const std::string &user_name,
             const char *protocol_name, int protocol_version);
 
@@ -145,7 +146,7 @@ class RpcEngine : public LockFreeRpcEngine, public std::enable_shared_from_this<
   const std::string &user_name() override { return auth_info_.getUser(); }
   const std::string &protocol_name() override { return protocol_name_; }
   int protocol_version() override { return protocol_version_; }
-  ::asio::io_service &io_service() override { return *io_service_; }
+  std::shared_ptr<IoService> io_service() const override { return io_service_; }
   const Options &options() override { return options_; }
   static std::string GetRandomClientName();
 
@@ -162,7 +163,7 @@ protected:
   std::vector<::asio::ip::tcp::endpoint> last_endpoints_;
 
 private:
-  ::asio::io_service * const io_service_;
+  mutable std::shared_ptr<IoService> io_service_;
   const Options options_;
   const std::string client_name_;
   const std::string client_id_;

+ 16 - 15
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc

@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
-#include "fs/filesystem.h"
-#include "fs/bad_datanode_tracker.h"
 #include "common/libhdfs_events_impl.h"
-
 #include "common/util.h"
+#include "fs/filesystem.h"
+#include "fs/filehandle.h"
+#include "fs/bad_datanode_tracker.h"
+#include "reader/block_reader.h"
 
 #include <gmock/gmock.h>
 
@@ -54,7 +55,7 @@ public:
     const std::string & client_name,
     const hadoop::hdfs::LocatedBlockProto &block,
     size_t offset,
-    const MutableBuffers &buffers,
+    const MutableBuffer &buffer,
     const std::function<void(const Status &, size_t)> handler));
 
   virtual void CancelOperation() override {
@@ -67,14 +68,14 @@ class MockDNConnection : public DataNodeConnection, public std::enable_shared_fr
       handler(Status::OK(), shared_from_this());
     }
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
         std::function<void (const asio::error_code & error,
                                std::size_t bytes_transferred) > handler) override {
       (void)buf;
       handler(asio::error::fault, 0);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
       (void)buf;
@@ -101,7 +102,7 @@ protected:
     return mock_reader_;
   }
   std::shared_ptr<DataNodeConnection> CreateDataNodeConnection(
-      ::asio::io_service *io_service,
+      std::shared_ptr<IoService> io_service,
       const ::hadoop::hdfs::DatanodeInfoProto & dn,
       const hadoop::common::TokenProto * token) override {
     (void) io_service; (void) dn; (void) token;
@@ -130,12 +131,12 @@ TEST(BadDataNodeTest, TestNoNodes) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto bad_node_tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
   bad_node_tracker->AddBadNode("foo");
 
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(), file_info, bad_node_tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(), file_info, bad_node_tracker, monitors);
   Status stat;
   size_t read = 0;
 
@@ -170,7 +171,7 @@ TEST(BadDataNodeTest, NNEventCallback) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
 
 
@@ -191,7 +192,7 @@ TEST(BadDataNodeTest, NNEventCallback) {
 
     return event_response::make_ok();
   });
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
 
@@ -234,10 +235,10 @@ TEST(BadDataNodeTest, RecoverableError) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
   EXPECT_CALL(*is.mock_reader_, AsyncReadBlock(_,_,_,_,_))
@@ -285,10 +286,10 @@ TEST(BadDataNodeTest, InternalError) {
   char buf[4096] = {
       0,
   };
-  IoServiceImpl io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   auto tracker = std::make_shared<BadDataNodeTracker>();
   auto monitors = std::make_shared<LibhdfsEvents>();
-  PartialMockFileHandle is("cluster", "file", &io_service.io_service(), GetRandomClientName(),  file_info, tracker, monitors);
+  PartialMockFileHandle is("cluster", "file", io_service, GetRandomClientName(),  file_info, tracker, monitors);
   Status stat;
   size_t read = 0;
   EXPECT_CALL(*is.mock_reader_, AsyncReadBlock(_,_,_,_,_))

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ioservice_test.cc

@@ -16,13 +16,15 @@
  * limitations under the License.
  */
 
-#include "common/hdfs_ioservice.h"
+#include "hdfspp/ioservice.h"
 
 #include <future>
 #include <functional>
 #include <thread>
 #include <string>
 
+
+#include <google/protobuf/stubs/common.h>
 #include <gmock/gmock.h>
 
 using ::testing::_;
@@ -34,7 +36,7 @@ using namespace hdfs;
 // Make sure IoService spins up specified number of threads
 TEST(IoServiceTest, InitThreads) {
 #ifndef DISABLE_CONCURRENT_WORKERS
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = 4;
@@ -50,7 +52,7 @@ TEST(IoServiceTest, InitThreads) {
 // Make sure IoService defaults to logical thread count
 TEST(IoServiceTest, InitDefaultThreads) {
 #ifndef DISABLE_CONCURRENT_WORKERS
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = std::thread::hardware_concurrency();
@@ -66,7 +68,7 @@ TEST(IoServiceTest, InitDefaultThreads) {
 
 // Check IoService::PostTask
 TEST(IoServiceTest, SimplePost) {
-  std::shared_ptr<IoServiceImpl> service = std::static_pointer_cast<IoServiceImpl>(IoService::MakeShared());
+  std::shared_ptr<IoService> service = IoService::MakeShared();
   EXPECT_NE(service, nullptr);
 
   unsigned int thread_count = std::thread::hardware_concurrency();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h

@@ -49,7 +49,7 @@ public:
   virtual ~MockConnectionBase();
   typedef std::pair<asio::error_code, std::string> ProducerResult;
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
           std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     if (produced_.size() == 0) {
@@ -72,7 +72,7 @@ public:
     io_service_->post(std::bind(handler, asio::error_code(), len));
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     // CompletionResult res = OnWrite(buf);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc

@@ -69,14 +69,14 @@ public:
   /* event handler to trigger side effects */
   std::function<void(void)> OnRead;
 
-  void async_read_some(const MutableBuffers &buf,
+  void async_read_some(const MutableBuffer &buf,
         std::function<void (const asio::error_code & error,
                                std::size_t bytes_transferred) > handler) override {
       this->OnRead();
       this->MockConnectionBase::async_read_some(buf, handler);
   }
 
-  void async_write_some(const ConstBuffers &buf,
+  void async_write_some(const ConstBuffer &buf,
             std::function<void (const asio::error_code & error,
                                  std::size_t bytes_transferred) > handler) override {
     this->MockConnectionBase::async_write_some(buf, handler);

+ 58 - 54
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc

@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include "hdfspp/ioservice.h"
+
 #include "mock_connection.h"
 #include "test.pb.h"
 #include "RpcHeader.pb.h"
@@ -23,7 +25,6 @@
 #include "common/namenode_info.h"
 
 #include <google/protobuf/io/coded_stream.h>
-
 #include <gmock/gmock.h>
 
 using ::hadoop::common::RpcResponseHeaderProto;
@@ -104,9 +105,10 @@ static inline std::pair<error_code, string> RpcResponse(
 using namespace hdfs;
 
 TEST(RpcEngineTest, TestRoundTrip) {
-  ::asio::io_service io_service;
+
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -129,20 +131,20 @@ TEST(RpcEngineTest, TestRoundTrip) {
   EchoRequestProto req;
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
-  engine->AsyncRpc("test", &req, resp, [resp, &complete,&io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [resp, &complete,io_service](const Status &stat) {
     ASSERT_TRUE(stat.ok());
     ASSERT_EQ("foo", resp->message());
     complete = true;
-    io_service.stop();
+    io_service->Stop();
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestConnectionResetAndFail) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -164,23 +166,23 @@ TEST(RpcEngineTest, TestConnectionResetAndFail) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 
 TEST(RpcEngineTest, TestConnectionResetAndRecover) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine
-      = std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      = std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -206,22 +208,22 @@ TEST(RpcEngineTest, TestConnectionResetAndRecover) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestConnectionResetAndRecoverWithDelay) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 1;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -246,17 +248,17 @@ TEST(RpcEngineTest, TestConnectionResetAndRecoverWithDelay) {
   req.set_message("foo");
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -267,7 +269,7 @@ TEST(RpcEngineTest, TestConnectionFailure)
   SharedMockConnection::SetSharedConnectionData(producer);
 
   // Error and no retry
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -275,16 +277,16 @@ TEST(RpcEngineTest, TestConnectionFailure)
   options.max_rpc_retries = 0;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine
-      = std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      = std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -294,7 +296,7 @@ TEST(RpcEngineTest, TestConnectionFailureRetryAndFailure)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -302,18 +304,18 @@ TEST(RpcEngineTest, TestConnectionFailureRetryAndFailure)
   options.max_rpc_retries = 2;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
@@ -323,7 +325,7 @@ TEST(RpcEngineTest, TestConnectionFailureAndRecover)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -331,29 +333,30 @@ TEST(RpcEngineTest, TestConnectionFailureAndRecover)
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(::asio::error_code(), "")))
       .WillOnce(Return(std::make_pair(::asio::error::would_block, "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestEventCallbacks)
 {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
+
   Options options;
   options.max_rpc_retries = 99;
   options.rpc_retry_delay_ms = 0;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
 
   // Normally determined during RpcEngine::Connect, but in this case options
   // provides enough info to determine policy here.
@@ -399,17 +402,18 @@ TEST(RpcEngineTest, TestEventCallbacks)
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
 
   bool complete = false;
-  engine->AsyncRpc("test", &req, resp, [&complete, &io_service](const Status &stat) {
+  engine->AsyncRpc("test", &req, resp, [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
   // If you're adding event hooks you'll most likely need to update this.
   // It's a brittle test but makes it hard to miss control flow changes in RPC retry.
-  for(const auto& m : callbacks)
+  for(const auto& m : callbacks) {
     std::cerr << m << std::endl;
-  io_service.run();
+  }
+  io_service->Run();
   ASSERT_TRUE(complete);
   ASSERT_EQ(9, callbacks.size());
   ASSERT_EQ(FS_NN_CONNECT_EVENT, callbacks[0]); // error
@@ -430,7 +434,7 @@ TEST(RpcEngineTest, TestConnectionFailureAndAsyncRecover)
   producer->checkProducerForConnect = true;
   SharedMockConnection::SetSharedConnectionData(producer);
 
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
 
   bool complete = false;
 
@@ -438,31 +442,31 @@ TEST(RpcEngineTest, TestConnectionFailureAndAsyncRecover)
   options.max_rpc_retries = 1;
   options.rpc_retry_delay_ms = 1;
   std::shared_ptr<SharedConnectionEngine> engine =
-      std::make_shared<SharedConnectionEngine>(&io_service, options, "foo", "", "protocol", 1);
+      std::make_shared<SharedConnectionEngine>(io_service, options, "foo", "", "protocol", 1);
   EXPECT_CALL(*producer, Produce())
       .WillOnce(Return(std::make_pair(make_error_code(::asio::error::connection_reset), "")))
       .WillOnce(Return(std::make_pair(::asio::error_code(), "")))
       .WillOnce(Return(std::make_pair(::asio::error::would_block, "")));
 
-  engine->Connect("", make_endpoint(), [&complete, &io_service](const Status &stat) {
+  engine->Connect("", make_endpoint(), [&complete, io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_TRUE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }
 
 TEST(RpcEngineTest, TestTimeout) {
-  ::asio::io_service io_service;
+  std::shared_ptr<IoService> io_service = IoService::MakeShared();
   Options options;
   options.rpc_timeout = 1;
-  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(&io_service, options, "foo", "", "protocol", 1);
+  std::shared_ptr<RpcEngine> engine = std::make_shared<RpcEngine>(io_service, options, "foo", "", "protocol", 1);
   auto conn =
       std::make_shared<RpcConnectionImpl<MockRPCConnection> >(engine);
   conn->TEST_set_connected(true);
@@ -481,15 +485,15 @@ TEST(RpcEngineTest, TestTimeout) {
   std::shared_ptr<EchoResponseProto> resp(new EchoResponseProto());
   engine->AsyncRpc("test", &req, resp, [resp, &complete,&io_service](const Status &stat) {
     complete = true;
-    io_service.stop();
+    io_service->Stop();
     ASSERT_FALSE(stat.ok());
   });
 
-  ::asio::deadline_timer timer(io_service);
+  ::asio::deadline_timer timer(io_service->GetRaw());
   timer.expires_from_now(std::chrono::hours(100));
   timer.async_wait([](const asio::error_code & err){(void)err; ASSERT_FALSE("Timed out"); });
 
-  io_service.run();
+  io_service->Run();
   ASSERT_TRUE(complete);
 }