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 2016/01/13 17:04:08 UTC

hadoop git commit: HDFS-9636. libhdfs++: for consistency, include files should be in hdfspp. Contributed by Bob Hansen.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-8707 1732e7f0a -> 5276e19a1


HDFS-9636.  libhdfs++: for consistency, include files should be in hdfspp.  Contributed by Bob Hansen.


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

Branch: refs/heads/HDFS-8707
Commit: 5276e19a18696f2c2fd414cc651a424ceb04d539
Parents: 1732e7f
Author: James <jh...@apache.org>
Authored: Wed Jan 13 11:03:29 2016 -0500
Committer: James <jh...@apache.org>
Committed: Wed Jan 13 11:03:29 2016 -0500

----------------------------------------------------------------------
 .../src/main/native/libhdfspp/CMakeLists.txt    |   4 +-
 .../native/libhdfspp/include/hdfspp/hdfs_ext.h  |  67 +++++++++
 .../native/libhdfspp/include/hdfspp/hdfspp.h    | 148 +++++++++++++++++++
 .../native/libhdfspp/include/hdfspp/options.h   |  57 +++++++
 .../native/libhdfspp/include/hdfspp/status.h    |  98 ++++++++++++
 .../libhdfspp/include/libhdfspp/hdfs_ext.h      |  67 ---------
 .../native/libhdfspp/include/libhdfspp/hdfspp.h | 148 -------------------
 .../libhdfspp/include/libhdfspp/options.h       |  57 -------
 .../native/libhdfspp/include/libhdfspp/status.h |  98 ------------
 .../native/libhdfspp/lib/bindings/c/hdfs.cc     |   2 +-
 .../libhdfspp/lib/common/continuation/asio.h    |   2 +-
 .../lib/common/continuation/continuation.h      |   2 +-
 .../libhdfspp/lib/common/hdfs_configuration.h   |   2 +-
 .../libhdfspp/lib/common/hdfs_public_api.h      |   2 +-
 .../main/native/libhdfspp/lib/common/options.cc |   2 +-
 .../libhdfspp/lib/common/sasl_authenticator.h   |   2 +-
 .../main/native/libhdfspp/lib/common/status.cc  |   2 +-
 .../src/main/native/libhdfspp/lib/common/util.h |   2 +-
 .../libhdfspp/lib/fs/bad_datanode_tracker.h     |   4 +-
 .../main/native/libhdfspp/lib/fs/filesystem.h   |   2 +-
 .../native/libhdfspp/lib/reader/block_reader.h  |   2 +-
 .../native/libhdfspp/lib/reader/datatransfer.cc |   2 +-
 .../main/native/libhdfspp/lib/rpc/rpc_engine.h  |   4 +-
 .../native/libhdfspp/tests/hdfspp_errors.cc     |   3 +-
 24 files changed, 389 insertions(+), 390 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
index d8e8c59..8475000 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt
@@ -165,8 +165,8 @@ set_target_properties(hdfspp PROPERTIES
 # Can be installed to a particular location via "make DESTDIR=... install"
 file(GLOB_RECURSE LIBHDFSPP_HEADER_FILES "${CMAKE_CURRENT_LIST_DIR}/include/*.h*")
 file(GLOB_RECURSE LIBHDFS_HEADER_FILES "${HADOOP_IMPORT_DIR}/include/*.h*")
-install(FILES ${LIBHDFSPP_HEADER_FILES} DESTINATION /include/libhdfspp)
-install(FILES ${LIBHDFS_HEADER_FILES} DESTINATION /include/libhdfs)
+install(FILES ${LIBHDFSPP_HEADER_FILES} DESTINATION /include/hdfspp)
+install(FILES ${LIBHDFS_HEADER_FILES} DESTINATION /include/hdfs)
 
 install(TARGETS hdfspp_static ARCHIVE DESTINATION /lib)
 install(TARGETS hdfspp LIBRARY DESTINATION /lib)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/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
new file mode 100644
index 0000000..2d793f4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfs_ext.h
@@ -0,0 +1,67 @@
+/**
+ * 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 LIBHDFSPP_HDFS_HDFSEXT
+#define LIBHDFSPP_HDFS_HDFSEXT
+
+/* get typdefs and #defines from libhdfs' hdfs.h to stay consistent */
+#include <hdfs/hdfs.h>
+
+/**
+ *  Note: The #defines below are copied directly from libhdfs'
+ *  hdfs.h.  LIBHDFS_EXTERNAL gets explicitly #undefed at the
+ *  end of the file so it must be redefined here.
+ **/
+
+#ifdef WIN32
+    #ifdef LIBHDFS_DLL_EXPORT
+        #define LIBHDFS_EXTERNAL __declspec(dllexport)
+    #elif LIBHDFS_DLL_IMPORT
+        #define LIBHDFS_EXTERNAL __declspec(dllimport)
+    #else
+        #define LIBHDFS_EXTERNAL
+    #endif
+#else
+    #ifdef LIBHDFS_DLL_EXPORT
+        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
+    #elif LIBHDFS_DLL_IMPORT
+        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
+    #else
+        #define LIBHDFS_EXTERNAL
+    #endif
+#endif
+
+
+/**
+ * Keep C bindings that are libhdfs++ specific in here.
+ **/
+
+extern "C" {
+/**
+ *  Reads the last error, if any, that happened in this thread
+ *  into the user supplied buffer.
+ *  @param buf  A chunk of memory with room for the error string.
+ *  @param len  Size of the buffer, if the message is longer than
+ *              len len-1 bytes of the message will be copied.
+ **/
+
+LIBHDFS_EXTERNAL
+void hdfsGetLastError(char *buf, int len);
+
+
+} /* end extern "C" */
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/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
new file mode 100644
index 0000000..effdecb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/hdfspp.h
@@ -0,0 +1,148 @@
+/**
+ * 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 LIBHDFSPP_HDFSPP_H_
+#define LIBHDFSPP_HDFSPP_H_
+
+#include "hdfspp/options.h"
+#include "hdfspp/status.h"
+
+#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
+ * file the operations.
+ *
+ * From an implementation point of view the IoService object wraps the
+ * ::asio::io_service objects. Please see the related documentation
+ * for more details.
+ **/
+class IoService {
+ public:
+  static IoService *New();
+  /**
+   * Run the asynchronous tasks associated with this IoService.
+   **/
+  virtual void Run() = 0;
+  /**
+   * Stop running asynchronous tasks associated with this IoService.
+   **/
+  virtual void Stop() = 0;
+  virtual ~IoService();
+};
+
+/**
+ * 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
+ * UUID.  The FileSystem and FileHandle use the BadDataNodeTracker
+ * by default.  AsyncPreadSome takes an optional NodeExclusionRule
+ * that will override the BadDataNodeTracker.
+ **/
+class NodeExclusionRule {
+ public:
+  virtual ~NodeExclusionRule(){};
+  virtual bool IsBadNode(const std::string &node_uuid) = 0;
+};
+
+/**
+ * Applications opens a FileHandle to read files in HDFS.
+ **/
+class FileHandle {
+public:
+  /**
+   * Read data from a specific position. The current implementation
+   * stops at the block boundary.
+   *
+   * @param buf the pointer to the buffer
+   * @param nbyte the size of the buffer
+   * @param offset the offset the file
+   *
+   * The handler returns the datanode that serves the block and the number of
+   * bytes has read.
+   **/
+  virtual void
+  PositionRead(void *buf, size_t nbyte, uint64_t offset,
+               const std::function<void(const Status &, size_t)> &handler) = 0;
+
+  virtual Status PositionRead(void *buf, size_t *nbyte, off_t offset) = 0;
+  virtual Status Read(void *buf, size_t *nbyte) = 0;
+  virtual Status Seek(off_t *offset, std::ios_base::seekdir whence) = 0;
+
+  /**
+   * Determine if a datanode should be excluded from future operations
+   * based on the return Status.
+   *
+   * @param status the Status object returned by InputStream::PositionRead
+   * @return true if the status indicates a failure that is not recoverable
+   * by the client and false otherwise.
+   **/
+  static bool ShouldExclude(const Status &status);
+
+  virtual ~FileHandle();
+};
+
+/**
+ * FileSystem implements APIs to interact with HDFS.
+ **/
+class FileSystem {
+ public:
+  /**
+   * Create a new instance of the FileSystem object. The call
+   * initializes the RPC connections to the NameNode and returns an
+   * FileSystem object.
+   **/
+  static FileSystem * New(
+      IoService *&io_service, const Options &options);
+
+  virtual void Connect(const std::string &server,
+      const std::string &service,
+      const std::function<void(const Status &, FileSystem *)> &&handler) = 0;
+
+  /* Synchronous call of Connect */
+  virtual Status Connect(const std::string &server,
+      const std::string &service) = 0;
+
+  /**
+   * Open a file on HDFS. The call issues an RPC to the NameNode to
+   * gather the locations of all blocks in the file and to return a
+   * new instance of the @ref InputStream object.
+   **/
+  virtual void
+  Open(const std::string &path,
+       const std::function<void(const Status &, FileHandle *)> &handler) = 0;
+  virtual Status Open(const std::string &path, FileHandle **handle) = 0;
+
+  /**
+   * Note that it is an error to destroy the filesystem from within a filesystem
+   * callback.  It will lead to a deadlock and the termination of the process.
+   */
+  virtual ~FileSystem() {};
+
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h
new file mode 100644
index 0000000..6a3799a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/options.h
@@ -0,0 +1,57 @@
+/**
+ * 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 LIBHDFSPP_OPTIONS_H_
+#define LIBHDFSPP_OPTIONS_H_
+
+namespace hdfs {
+
+/**
+ * Options to control the behavior of the libhdfspp library.
+ **/
+struct Options {
+  /**
+   * Time out of RPC requests in milliseconds.
+   * Default: 30000
+   **/
+  int rpc_timeout;
+  static const int kDefaultRpcTimeout = 30000;
+
+  /**
+   * Maximum number of retries for RPC operations
+   **/
+  int max_rpc_retries;
+  static const int kNoRetry = -1;
+  static const int kDefaultMaxRpcRetries = kNoRetry;
+
+  /**
+   * Number of ms to wait between retry of RPC operations
+   **/
+  int rpc_retry_delay_ms;
+  static const int kDefaultRpcRetryDelayMs = 10000;
+
+  /**
+   * Exclusion time for failed datanodes in milliseconds.
+   * Default: 60000
+   **/
+  unsigned int host_exclusion_duration;
+  static const unsigned int kDefaultHostExclusionDuration = 600000;
+
+  Options();
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
new file mode 100644
index 0000000..6b58799
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/hdfspp/status.h
@@ -0,0 +1,98 @@
+/**
+ * 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 LIBHDFSPP_STATUS_H_
+#define LIBHDFSPP_STATUS_H_
+
+#include <string>
+#include <system_error>
+
+namespace hdfs {
+
+class StatusHelper;
+class Status {
+ public:
+  // Create a success status.
+  Status() : state_(NULL) { }
+  ~Status() { delete[] state_; }
+  explicit Status(int code, const char *msg);
+
+  // Copy the specified status.
+  Status(const Status& s);
+  void operator=(const Status& s);
+
+  // Return a success status.
+  static Status OK() { return Status(); }
+  static Status InvalidArgument(const char *msg)
+  { return Status(kInvalidArgument, msg); }
+  static Status ResourceUnavailable(const char *msg)
+  { return Status(kResourceUnavailable, msg); }
+  static Status Unimplemented()
+  { return Status(kUnimplemented, ""); }
+  static Status Exception(const char *expception_class_name, const char *error_message)
+  { return Status(kException, expception_class_name, error_message); }
+  static Status Error(const char *error_message)
+  { return Exception("Exception", error_message); }
+
+  // Returns true iff the status indicates success.
+  bool ok() const { return (state_ == NULL); }
+
+  // Return a string representation of this status suitable for printing.
+  // Returns the string "OK" for success.
+  std::string ToString() const;
+
+  int code() const {
+    return (state_ == NULL) ? kOk : static_cast<int>(state_[4]);
+  }
+
+  enum Code {
+    kOk = 0,
+    kInvalidArgument = static_cast<unsigned>(std::errc::invalid_argument),
+    kResourceUnavailable = static_cast<unsigned>(std::errc::resource_unavailable_try_again),
+    kUnimplemented = static_cast<unsigned>(std::errc::function_not_supported),
+    kException = 255,
+  };
+
+ private:
+  // OK status has a NULL state_.  Otherwise, state_ is a new[] array
+  // of the following form:
+  //    state_[0..3] == length of message
+  //    state_[4]    == code
+  //    state_[5..]  == message
+  const char* state_;
+
+  explicit Status(int code, const char *msg1, const char *msg2);
+  static const char *CopyState(const char* s);
+  static const char *ConstructState(int code, const char *msg1, const char *msg2);
+};
+
+inline Status::Status(const Status& s) {
+  state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+}
+
+inline void Status::operator=(const Status& s) {
+  // The following condition catches both aliasing (when this == &s),
+  // and the common case where both s and *this are ok.
+  if (state_ != s.state_) {
+    delete[] state_;
+    state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+  }
+}
+
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
deleted file mode 100644
index 2d793f4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfs_ext.h
+++ /dev/null
@@ -1,67 +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.
- */
-#ifndef LIBHDFSPP_HDFS_HDFSEXT
-#define LIBHDFSPP_HDFS_HDFSEXT
-
-/* get typdefs and #defines from libhdfs' hdfs.h to stay consistent */
-#include <hdfs/hdfs.h>
-
-/**
- *  Note: The #defines below are copied directly from libhdfs'
- *  hdfs.h.  LIBHDFS_EXTERNAL gets explicitly #undefed at the
- *  end of the file so it must be redefined here.
- **/
-
-#ifdef WIN32
-    #ifdef LIBHDFS_DLL_EXPORT
-        #define LIBHDFS_EXTERNAL __declspec(dllexport)
-    #elif LIBHDFS_DLL_IMPORT
-        #define LIBHDFS_EXTERNAL __declspec(dllimport)
-    #else
-        #define LIBHDFS_EXTERNAL
-    #endif
-#else
-    #ifdef LIBHDFS_DLL_EXPORT
-        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
-    #elif LIBHDFS_DLL_IMPORT
-        #define LIBHDFS_EXTERNAL __attribute__((visibility("default")))
-    #else
-        #define LIBHDFS_EXTERNAL
-    #endif
-#endif
-
-
-/**
- * Keep C bindings that are libhdfs++ specific in here.
- **/
-
-extern "C" {
-/**
- *  Reads the last error, if any, that happened in this thread
- *  into the user supplied buffer.
- *  @param buf  A chunk of memory with room for the error string.
- *  @param len  Size of the buffer, if the message is longer than
- *              len len-1 bytes of the message will be copied.
- **/
-
-LIBHDFS_EXTERNAL
-void hdfsGetLastError(char *buf, int len);
-
-
-} /* end extern "C" */
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
deleted file mode 100644
index ebcf227..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/hdfspp.h
+++ /dev/null
@@ -1,148 +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.
- */
-#ifndef LIBHDFSPP_HDFSPP_H_
-#define LIBHDFSPP_HDFSPP_H_
-
-#include "libhdfspp/options.h"
-#include "libhdfspp/status.h"
-
-#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
- * file the operations.
- *
- * From an implementation point of view the IoService object wraps the
- * ::asio::io_service objects. Please see the related documentation
- * for more details.
- **/
-class IoService {
- public:
-  static IoService *New();
-  /**
-   * Run the asynchronous tasks associated with this IoService.
-   **/
-  virtual void Run() = 0;
-  /**
-   * Stop running asynchronous tasks associated with this IoService.
-   **/
-  virtual void Stop() = 0;
-  virtual ~IoService();
-};
-
-/**
- * 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
- * UUID.  The FileSystem and FileHandle use the BadDataNodeTracker
- * by default.  AsyncPreadSome takes an optional NodeExclusionRule
- * that will override the BadDataNodeTracker.
- **/
-class NodeExclusionRule {
- public:
-  virtual ~NodeExclusionRule(){};
-  virtual bool IsBadNode(const std::string &node_uuid) = 0;
-};
-
-/**
- * Applications opens a FileHandle to read files in HDFS.
- **/
-class FileHandle {
-public:
-  /**
-   * Read data from a specific position. The current implementation
-   * stops at the block boundary.
-   *
-   * @param buf the pointer to the buffer
-   * @param nbyte the size of the buffer
-   * @param offset the offset the file
-   *
-   * The handler returns the datanode that serves the block and the number of
-   * bytes has read.
-   **/
-  virtual void
-  PositionRead(void *buf, size_t nbyte, uint64_t offset,
-               const std::function<void(const Status &, size_t)> &handler) = 0;
-
-  virtual Status PositionRead(void *buf, size_t *nbyte, off_t offset) = 0;
-  virtual Status Read(void *buf, size_t *nbyte) = 0;
-  virtual Status Seek(off_t *offset, std::ios_base::seekdir whence) = 0;
-
-  /**
-   * Determine if a datanode should be excluded from future operations
-   * based on the return Status.
-   *
-   * @param status the Status object returned by InputStream::PositionRead
-   * @return true if the status indicates a failure that is not recoverable
-   * by the client and false otherwise.
-   **/
-  static bool ShouldExclude(const Status &status);
-
-  virtual ~FileHandle();
-};
-
-/**
- * FileSystem implements APIs to interact with HDFS.
- **/
-class FileSystem {
- public:
-  /**
-   * Create a new instance of the FileSystem object. The call
-   * initializes the RPC connections to the NameNode and returns an
-   * FileSystem object.
-   **/
-  static FileSystem * New(
-      IoService *&io_service, const Options &options);
-
-  virtual void Connect(const std::string &server,
-      const std::string &service,
-      const std::function<void(const Status &, FileSystem *)> &&handler) = 0;
-
-  /* Synchronous call of Connect */
-  virtual Status Connect(const std::string &server,
-      const std::string &service) = 0;
-
-  /**
-   * Open a file on HDFS. The call issues an RPC to the NameNode to
-   * gather the locations of all blocks in the file and to return a
-   * new instance of the @ref InputStream object.
-   **/
-  virtual void
-  Open(const std::string &path,
-       const std::function<void(const Status &, FileHandle *)> &handler) = 0;
-  virtual Status Open(const std::string &path, FileHandle **handle) = 0;
-
-  /**
-   * Note that it is an error to destroy the filesystem from within a filesystem
-   * callback.  It will lead to a deadlock and the termination of the process.
-   */
-  virtual ~FileSystem() {};
-
-};
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h
deleted file mode 100644
index 6a3799a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/options.h
+++ /dev/null
@@ -1,57 +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.
- */
-#ifndef LIBHDFSPP_OPTIONS_H_
-#define LIBHDFSPP_OPTIONS_H_
-
-namespace hdfs {
-
-/**
- * Options to control the behavior of the libhdfspp library.
- **/
-struct Options {
-  /**
-   * Time out of RPC requests in milliseconds.
-   * Default: 30000
-   **/
-  int rpc_timeout;
-  static const int kDefaultRpcTimeout = 30000;
-
-  /**
-   * Maximum number of retries for RPC operations
-   **/
-  int max_rpc_retries;
-  static const int kNoRetry = -1;
-  static const int kDefaultMaxRpcRetries = kNoRetry;
-
-  /**
-   * Number of ms to wait between retry of RPC operations
-   **/
-  int rpc_retry_delay_ms;
-  static const int kDefaultRpcRetryDelayMs = 10000;
-
-  /**
-   * Exclusion time for failed datanodes in milliseconds.
-   * Default: 60000
-   **/
-  unsigned int host_exclusion_duration;
-  static const unsigned int kDefaultHostExclusionDuration = 600000;
-
-  Options();
-};
-}
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/status.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/status.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/status.h
deleted file mode 100644
index 6b58799..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/include/libhdfspp/status.h
+++ /dev/null
@@ -1,98 +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.
- */
-#ifndef LIBHDFSPP_STATUS_H_
-#define LIBHDFSPP_STATUS_H_
-
-#include <string>
-#include <system_error>
-
-namespace hdfs {
-
-class StatusHelper;
-class Status {
- public:
-  // Create a success status.
-  Status() : state_(NULL) { }
-  ~Status() { delete[] state_; }
-  explicit Status(int code, const char *msg);
-
-  // Copy the specified status.
-  Status(const Status& s);
-  void operator=(const Status& s);
-
-  // Return a success status.
-  static Status OK() { return Status(); }
-  static Status InvalidArgument(const char *msg)
-  { return Status(kInvalidArgument, msg); }
-  static Status ResourceUnavailable(const char *msg)
-  { return Status(kResourceUnavailable, msg); }
-  static Status Unimplemented()
-  { return Status(kUnimplemented, ""); }
-  static Status Exception(const char *expception_class_name, const char *error_message)
-  { return Status(kException, expception_class_name, error_message); }
-  static Status Error(const char *error_message)
-  { return Exception("Exception", error_message); }
-
-  // Returns true iff the status indicates success.
-  bool ok() const { return (state_ == NULL); }
-
-  // Return a string representation of this status suitable for printing.
-  // Returns the string "OK" for success.
-  std::string ToString() const;
-
-  int code() const {
-    return (state_ == NULL) ? kOk : static_cast<int>(state_[4]);
-  }
-
-  enum Code {
-    kOk = 0,
-    kInvalidArgument = static_cast<unsigned>(std::errc::invalid_argument),
-    kResourceUnavailable = static_cast<unsigned>(std::errc::resource_unavailable_try_again),
-    kUnimplemented = static_cast<unsigned>(std::errc::function_not_supported),
-    kException = 255,
-  };
-
- private:
-  // OK status has a NULL state_.  Otherwise, state_ is a new[] array
-  // of the following form:
-  //    state_[0..3] == length of message
-  //    state_[4]    == code
-  //    state_[5..]  == message
-  const char* state_;
-
-  explicit Status(int code, const char *msg1, const char *msg2);
-  static const char *CopyState(const char* s);
-  static const char *ConstructState(int code, const char *msg1, const char *msg2);
-};
-
-inline Status::Status(const Status& s) {
-  state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
-}
-
-inline void Status::operator=(const Status& s) {
-  // The following condition catches both aliasing (when this == &s),
-  // and the common case where both s and *this are ok.
-  if (state_ != s.state_) {
-    delete[] state_;
-    state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
-  }
-}
-
-}
-
-#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/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 0fc02b4..1d96d76 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
@@ -19,7 +19,7 @@
 #include "fs/filesystem.h"
 
 #include <hdfs/hdfs.h>
-#include <libhdfspp/hdfs_ext.h>
+#include <hdfspp/hdfs_ext.h>
 
 #include <string>
 #include <cstring>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
index a5a0446..039daf7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/asio.h
@@ -21,7 +21,7 @@
 #include "continuation.h"
 #include "common/util.h"
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 #include <asio/connect.hpp>
 #include <asio/read.hpp>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/continuation.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/continuation.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/continuation.h
index 0af04a8..58a1b46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/continuation.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/continuation/continuation.h
@@ -18,7 +18,7 @@
 #ifndef LIB_COMMON_CONTINUATION_CONTINUATION_H_
 #define LIB_COMMON_CONTINUATION_CONTINUATION_H_
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 #include <functional>
 #include <memory>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h
index ce456d8..182cacd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_configuration.h
@@ -20,7 +20,7 @@
 #define COMMON_HDFS_CONFIGURATION_H_
 
 #include "common/configuration.h"
-#include "libhdfspp/options.h"
+#include "hdfspp/options.h"
 
 #include <string>
 #include <map>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
index acd96024..1620f62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/hdfs_public_api.h
@@ -19,7 +19,7 @@
 #ifndef COMMON_HDFS_PUBLIC_API_H_
 #define COMMON_HDFS_PUBLIC_API_H_
 
-#include "libhdfspp/hdfspp.h"
+#include "hdfspp/hdfspp.h"
 
 #include <asio/io_service.hpp>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
index 4e8dfbc..c7d418f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/options.cc
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "libhdfspp/options.h"
+#include "hdfspp/options.h"
 
 namespace hdfs {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/sasl_authenticator.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/sasl_authenticator.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/sasl_authenticator.h
index 71fee7a..78b2a55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/sasl_authenticator.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/sasl_authenticator.h
@@ -18,7 +18,7 @@
 #ifndef LIB_COMMON_SASL_AUTHENTICATOR_H_
 #define LIB_COMMON_SASL_AUTHENTICATOR_H_
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 namespace hdfs {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
index 66cfa1c..828f6aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/status.cc
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 #include <cassert>
 #include <cstring>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
index 391d4c7..60f70e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.h
@@ -18,7 +18,7 @@
 #ifndef LIB_COMMON_UTIL_H_
 #define LIB_COMMON_UTIL_H_
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 #include <sstream>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
index 72855dc..1d596ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/bad_datanode_tracker.h
@@ -25,8 +25,8 @@
 #include <string>
 #include <set>
 
-#include "libhdfspp/options.h"
-#include "libhdfspp/hdfspp.h"
+#include "hdfspp/options.h"
+#include "hdfspp/hdfspp.h"
 
 namespace hdfs {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
index df61872..0ae032d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.h
@@ -21,7 +21,7 @@
 #include "filehandle.h"
 #include "common/hdfs_public_api.h"
 #include "common/async_stream.h"
-#include "libhdfspp/hdfspp.h"
+#include "hdfspp/hdfspp.h"
 #include "fs/bad_datanode_tracker.h"
 #include "rpc/rpc_engine.h"
 #include "reader/block_reader.h"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
index 140286b..7984c7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.h
@@ -18,7 +18,7 @@
 #ifndef BLOCK_READER_H_
 #define BLOCK_READER_H_
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 #include "common/async_stream.h"
 #include "datatransfer.pb.h"
 #include "connection/datanodeconnection.h"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.cc
index d936407..c3cbf7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.cc
@@ -18,7 +18,7 @@
 
 #include "datatransfer.h"
 
-#include "libhdfspp/status.h"
+#include "hdfspp/status.h"
 
 namespace hdfs {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
index e6beef6..fb1844d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.h
@@ -18,8 +18,8 @@
 #ifndef LIB_RPC_RPC_ENGINE_H_
 #define LIB_RPC_RPC_ENGINE_H_
 
-#include "libhdfspp/options.h"
-#include "libhdfspp/status.h"
+#include "hdfspp/options.h"
+#include "hdfspp/status.h"
 
 #include "common/retry_policy.h"
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5276e19a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
index 0cb6545..f49f5fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_errors.cc
@@ -18,7 +18,7 @@
 
 
 #include <hdfs/hdfs.h>
-#include <libhdfspp/hdfs_ext.h>
+#include <hdfspp/hdfs_ext.h>
 
 #include <google/protobuf/io/coded_stream.h>
 #include <gmock/gmock.h>
@@ -114,4 +114,3 @@ int main(int argc, char *argv[]) {
 
   return exit_code;
 }
-