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 ga...@apache.org on 2022/07/23 17:50:24 UTC

[hadoop] branch trunk updated: HDFS-16467. Ensure Protobuf generated headers are included first (#4601)

This is an automated email from the ASF dual-hosted git repository.

gaurava pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 7de9b5ee27a HDFS-16467. Ensure Protobuf generated headers are included first (#4601)
7de9b5ee27a is described below

commit 7de9b5ee27a19f948fe88599e21e4a481cf33410
Author: Gautham B A <ga...@gmail.com>
AuthorDate: Sat Jul 23 23:20:15 2022 +0530

    HDFS-16467. Ensure Protobuf generated headers are included first (#4601)
    
    * This PR ensures that the Protobuf generated headers
      are always included first, even when these headers
      are included transitively.
    * This problem is specific to Windows only.
---
 .../main/native/libhdfs-tests/test_libhdfs_mini_stress.c  | 15 +++++++++------
 .../src/main/native/libhdfs/os/windows/unistd.h           |  2 +-
 .../src/main/native/libhdfspp/lib/bindings/c/hdfs.cc      |  3 ++-
 .../src/main/native/libhdfspp/lib/common/fsinfo.cc        |  1 +
 .../src/main/native/libhdfspp/lib/common/logging.h        |  4 ++--
 .../src/main/native/libhdfspp/lib/common/namenode_info.cc |  1 +
 .../src/main/native/libhdfspp/lib/common/util.cc          |  1 +
 .../native/libhdfspp/lib/connection/datanodeconnection.h  |  3 ++-
 .../src/main/native/libhdfspp/lib/fs/filehandle.cc        |  3 ++-
 .../src/main/native/libhdfspp/lib/fs/filehandle.h         |  3 ++-
 .../src/main/native/libhdfspp/lib/fs/filesystem.cc        |  9 ++++++---
 .../main/native/libhdfspp/lib/fs/namenode_operations.cc   |  5 ++++-
 .../main/native/libhdfspp/lib/fs/namenode_operations.h    |  6 ++++--
 .../src/main/native/libhdfspp/lib/reader/block_reader.cc  |  2 ++
 .../src/main/native/libhdfspp/lib/reader/block_reader.h   |  3 ++-
 .../src/main/native/libhdfspp/lib/reader/datatransfer.h   |  2 ++
 .../main/native/libhdfspp/lib/reader/datatransfer_impl.h  |  5 ++++-
 .../src/main/native/libhdfspp/lib/reader/readergroup.h    |  3 ++-
 .../src/main/native/libhdfspp/lib/rpc/request.cc          |  9 +++++----
 .../src/main/native/libhdfspp/lib/rpc/rpc_connection.h    |  1 +
 .../main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc  |  8 +++++---
 .../main/native/libhdfspp/lib/rpc/rpc_connection_impl.h   |  1 +
 .../src/main/native/libhdfspp/lib/rpc/rpc_engine.cc       |  1 +
 .../src/main/native/libhdfspp/lib/rpc/sasl_protocol.cc    |  2 +-
 .../src/main/native/libhdfspp/lib/rpc/sasl_protocol.h     |  4 ++--
 .../src/main/native/libhdfspp/tests/bad_datanode_test.cc  |  5 +++--
 .../src/main/native/libhdfspp/tests/hdfs_shim.c           |  4 ++++
 .../src/main/native/libhdfspp/tests/logging_test.cc       |  4 ++--
 .../src/main/native/libhdfspp/tests/mock_connection.cc    |  2 ++
 .../src/main/native/libhdfspp/tests/mock_connection.h     |  1 +
 .../native/libhdfspp/tests/remote_block_reader_test.cc    | 14 +++++++-------
 .../src/main/native/libhdfspp/tests/rpc_engine_test.cc    |  6 ++++--
 32 files changed, 88 insertions(+), 45 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
index 1641470733f..7318aaafbae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_mini_stress.c
@@ -18,24 +18,27 @@
 
 #include "common/util_c.h"
 #include "expect.h"
-#include "hdfs/hdfs.h"
 #include "hdfspp/hdfs_ext.h"
 #include "native_mini_dfs.h"
 #include "os/thread.h"
 #include "x-platform/c-api/syscall.h"
+#include "hdfs/hdfs.h"
 
 #include <errno.h>
 #include <inttypes.h>
-#include <pwd.h>
 #include <stdint.h>
 #include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
-#include <sys/socket.h>
 #include <sys/types.h>
-#include <sys/wait.h>
 #include <unistd.h>
 
+#ifndef WIN32
+#include <sys/socket.h>
+#include <sys/wait.h>
+#include <pwd.h>
+#endif
+
 #define TO_STR_HELPER(X) #X
 #define TO_STR(X) TO_STR_HELPER(X)
 
@@ -197,7 +200,7 @@ static int fileEventCallback1(const char * event, const char * cluster, const ch
   if (randomErrRatioStr) randomErrRatio = (int64_t)atoi(randomErrRatioStr);
   if (randomErrRatio == 0) return DEBUG_SIMULATE_ERROR;
   else if (randomErrRatio < 0) return LIBHDFSPP_EVENT_OK;
-  return random() % randomErrRatio == 0 ? DEBUG_SIMULATE_ERROR : LIBHDFSPP_EVENT_OK;
+  return rand() % randomErrRatio == 0 ? DEBUG_SIMULATE_ERROR : LIBHDFSPP_EVENT_OK;
 }
 
 static int fileEventCallback2(const char * event, const char * cluster, const char * file, int64_t value, int64_t cookie)
@@ -235,7 +238,7 @@ static int doTestHdfsMiniStress(struct tlhThreadInfo *ti, int randomErr)
     EXPECT_ZERO(hdfsCloseFile(ti->hdfs, file));
     file = hdfsOpenFile(ti->hdfs, ti->fileNm, O_RDONLY, 0, 0, 0);
     EXPECT_NONNULL(file);
-    seekPos = (((double)random()) / RAND_MAX) * (fileInfo->mSize - expected);
+    seekPos = (((double)rand()) / RAND_MAX) * (fileInfo->mSize - expected);
     seekPos = (seekPos / expected) * expected;
     ret = hdfsSeek(ti->hdfs, file, seekPos);
     if (ret < 0) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/unistd.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/unistd.h
index b82ce48968d..900264240e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/unistd.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/os/windows/unistd.h
@@ -22,7 +22,7 @@
 /* On Windows, unistd.h does not exist, so manually define what we need. */
 
 #include <process.h> /* Declares getpid(). */
-#include <windows.h>
+#include <Windows.h>
 
 /* Re-route sleep to Sleep, converting units from seconds to milliseconds. */
 #define sleep(seconds) Sleep((seconds) * 1000)
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 80f93161602..9386453c929 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
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include "fs/filehandle.h"
+
 #include "hdfspp/hdfspp.h"
 #include "hdfspp/hdfs_ext.h"
 
@@ -23,7 +25,6 @@
 #include "common/configuration_loader.h"
 #include "common/logging.h"
 #include "fs/filesystem.h"
-#include "fs/filehandle.h"
 #include "x-platform/utils.h"
 #include "x-platform/syscall.h"
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/fsinfo.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/fsinfo.cc
index f8f59238327..ec7efc8f270 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/fsinfo.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/fsinfo.cc
@@ -21,6 +21,7 @@
 #include <algorithm>
 #include <sstream>
 #include <iomanip>
+#include <algorithm>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h
index 8935287fe01..9d75a48791d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/logging.h
@@ -19,10 +19,10 @@
 #ifndef LIB_COMMON_LOGGING_H_
 #define LIB_COMMON_LOGGING_H_
 
-#include <boost/asio/ip/tcp.hpp>
-
 #include "hdfspp/log.h"
 
+#include <boost/asio/ip/tcp.hpp>
+
 #include <sstream>
 #include <mutex>
 #include <memory>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
index 92054fce07e..4b375711f74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/namenode_info.cc
@@ -26,6 +26,7 @@
 #include <utility>
 #include <future>
 
+#include <boost/system/error_code.hpp>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc
index c0e10183297..45798bdd8e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/util.cc
@@ -19,6 +19,7 @@
 #include "common/util.h"
 #include "common/util_c.h"
 
+#include <boost/system/error_code.hpp>
 #include <google/protobuf/message_lite.h>
 #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h
index a0cb8375a86..b5fe7b975c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/connection/datanodeconnection.h
@@ -18,9 +18,10 @@
 #ifndef LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 #define LIBHDFSPP_LIB_CONNECTION_DATANODECONNECTION_H_
 
+#include "ClientNamenodeProtocol.pb.h"
+
 #include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
-#include "ClientNamenodeProtocol.pb.h"
 #include "common/libhdfs_events_impl.h"
 #include "common/logging.h"
 #include "common/util.h"
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc
index 7cfd6df3b98..d5118089027 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.cc
@@ -16,11 +16,12 @@
  * limitations under the License.
  */
 
+#include "reader/block_reader.h"
 #include "filehandle.h"
+
 #include "common/continuation/continuation.h"
 #include "common/logging.h"
 #include "connection/datanodeconnection.h"
-#include "reader/block_reader.h"
 #include "hdfspp/events.h"
 #include "x-platform/types.h"
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h
index 0e4eed7af4e..cfff9b1b3f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filehandle.h
@@ -18,13 +18,14 @@
 #ifndef LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 #define LIBHDFSPP_LIB_FS_FILEHANDLE_H_
 
+#include "reader/readergroup.h"
+
 #include "hdfspp/ioservice.h"
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
 #include "common/libhdfs_events_impl.h"
 #include "common/new_delete.h"
 #include "reader/fileinfo.h"
-#include "reader/readergroup.h"
 
 #include "bad_datanode_tracker.h"
 #include "ClientNamenodeProtocol.pb.h"
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
index e92a9ee48d6..e2b6cea30b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/filesystem.cc
@@ -16,21 +16,24 @@
  * limitations under the License.
  */
 
-#include "filesystem.h"
-
 #include "filehandle.h"
+#include "filesystem.h"
 #include "common/namenode_info.h"
 
 #include <functional>
 #include <limits>
 #include <future>
 #include <tuple>
-#include <pwd.h>
+#include <iostream>
 
 #include <boost/asio/ip/tcp.hpp>
 
 #include "x-platform/syscall.h"
 
+#ifndef WIN32
+#include <pwd.h>
+#endif
+
 #define FMT_THIS_ADDR "this=" << (void*)this
 
 namespace hdfs {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
index 96744e5d03d..4c9e398286c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.cc
@@ -26,9 +26,12 @@
 #include <future>
 #include <tuple>
 #include <iostream>
-#include <pwd.h>
 #include <utility>
 
+#ifndef WIN32
+#include <pwd.h>
+#endif
+
 #define FMT_THIS_ADDR "this=" << (void*)this
 
 using boost::asio::ip::tcp;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
index 445aa08653d..bcccd52a56d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/fs/namenode_operations.h
@@ -18,17 +18,19 @@
 #ifndef LIBHDFSPP_LIB_FS_NAMENODEOPERATIONS_H_
 #define LIBHDFSPP_LIB_FS_NAMENODEOPERATIONS_H_
 
+#include "ClientNamenodeProtocol.pb.h"
+
 #include "rpc/rpc_engine.h"
 #include "hdfspp/statinfo.h"
 #include "hdfspp/fsinfo.h"
 #include "hdfspp/content_summary.h"
 #include "common/namenode_info.h"
-#include "ClientNamenodeProtocol.pb.h"
-#include "ClientNamenodeProtocol.hrpc.inl"
 
 #include <memory>
 #include <string>
 
+#include "ClientNamenodeProtocol.hrpc.inl"
+
 namespace hdfs {
 
 /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc
index acecfce5237..8db645e48d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/block_reader.cc
@@ -15,8 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 #include "reader/block_reader.h"
 #include "reader/datatransfer.h"
+
 #include "common/continuation/continuation.h"
 #include "common/continuation/asio.h"
 #include "common/logging.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 167c57d3a17..605d641b3b3 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,11 +18,12 @@
 #ifndef BLOCK_READER_H_
 #define BLOCK_READER_H_
 
+#include "datatransfer.pb.h"
+
 #include "hdfspp/status.h"
 #include "common/async_stream.h"
 #include "common/cancel_tracker.h"
 #include "common/new_delete.h"
-#include "datatransfer.pb.h"
 #include "connection/datanodeconnection.h"
 
 #include <memory>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h
index cfa94bea2ba..025cb8d3708 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer.h
@@ -18,6 +18,8 @@
 #ifndef LIB_READER_DATA_TRANSFER_H_
 #define LIB_READER_DATA_TRANSFER_H_
 
+#include "datatransfer.pb.h"
+
 #include "common/sasl_authenticator.h"
 #include "common/async_stream.h"
 #include "connection/datanodeconnection.h"
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h
index d77685dd45a..e3249ece51e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/datatransfer_impl.h
@@ -18,10 +18,13 @@
 #ifndef LIB_READER_DATATRANFER_IMPL_H_
 #define LIB_READER_DATATRANFER_IMPL_H_
 
-#include "datatransfer.pb.h"
 #include "common/continuation/continuation.h"
 #include "common/continuation/asio.h"
 #include "common/continuation/protobuf.h"
+#include "common/sasl_authenticator.h"
+
+#include <boost/asio/read.hpp>
+#include <boost/asio/buffer.hpp>
 
 #include <boost/asio/read.hpp>
 #include <boost/asio/buffer.hpp>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/readergroup.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/readergroup.h
index e6173f7fa70..47b6046d80f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/readergroup.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/reader/readergroup.h
@@ -15,14 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 #ifndef READER_READER_GROUP_H_
 #define READER_READER_GROUP_H_
 
 #include "block_reader.h"
 
 #include <memory>
-#include <vector>
 #include <mutex>
+#include <vector>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc
index 99762c89ee9..bf25e60c68c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/request.cc
@@ -16,16 +16,17 @@
  * limitations under the License.
  */
 
+#include "RpcHeader.pb.h"
+#include "ProtobufRpcEngine.pb.h"
+#include "IpcConnectionContext.pb.h"
+
 #include <functional>
+
 #include "request.h"
 #include "rpc_engine.h"
 #include "sasl_protocol.h"
 #include "hdfspp/ioservice.h"
 
-#include "RpcHeader.pb.h"
-#include "ProtobufRpcEngine.pb.h"
-#include "IpcConnectionContext.pb.h"
-
 #include <sstream>
 
 namespace hdfs {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h
index f599d36ee5c..21c475c1e8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection.h
@@ -43,6 +43,7 @@
 #include <unordered_map>
 
 #include <boost/asio/ip/tcp.hpp>
+#include <boost/system/error_code.hpp>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc
index 5d434ef370a..2a4e358e9bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.cc
@@ -15,15 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-#include "rpc_engine.h"
-#include "rpc_connection_impl.h"
-#include "sasl_protocol.h"
 
 #include "RpcHeader.pb.h"
 #include "ProtobufRpcEngine.pb.h"
 #include "IpcConnectionContext.pb.h"
 
+#include "rpc_engine.h"
+#include "rpc_connection_impl.h"
+#include "sasl_protocol.h"
+
 #include <boost/asio/error.hpp>
+#include <boost/system/error_code.hpp>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h
index 884bd64ac64..4f90ab72f99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_connection_impl.h
@@ -35,6 +35,7 @@
 #include <boost/date_time/posix_time/posix_time_duration.hpp>
 
 #include <system_error>
+#include <boost/system/error_code.hpp>
 
 namespace hdfs {
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc
index e3274cb88aa..0bea5bf192c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/rpc_engine.cc
@@ -27,6 +27,7 @@
 #include <string>
 
 #include <boost/date_time/posix_time/posix_time_duration.hpp>
+#include <boost/system/error_code.hpp>
 #include <openssl/rand.h>
 #include <openssl/err.h>
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.cc
index bc9adbff313..c45ab84d213 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.cc
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include "sasl_protocol.h"
 #include "rpc_engine.h"
 #include "rpc_connection.h"
 #include "common/logging.h"
@@ -23,7 +24,6 @@
 #include "x-platform/syscall.h"
 
 #include "sasl_engine.h"
-#include "sasl_protocol.h"
 
 #if defined USE_SASL
   #if defined USE_CYRUS_SASL
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.h
index a46ae08074d..2c7fd09b96b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/sasl_protocol.h
@@ -19,12 +19,12 @@
 #ifndef LIB_RPC_SASLPROTOCOL_H
 #define LIB_RPC_SASLPROTOCOL_H
 
+#include "RpcHeader.pb.h"
+
 #include <memory>
 #include <mutex>
 #include <functional>
 
-#include <RpcHeader.pb.h>
-
 #include "hdfspp/status.h"
 #include "common/auth_info.h"
 #include "common/libhdfs_events_impl.h"
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc
index 973212647e7..225dd5e20ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/bad_datanode_test.cc
@@ -16,12 +16,13 @@
  * limitations under the License.
  */
 
+#include "reader/block_reader.h"
+#include "fs/filehandle.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 <gtest/gtest.h>
 #include <gmock/gmock.h>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
index 2d265b8f03c..ad8ad712c9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c
@@ -24,6 +24,10 @@
 #include <stdlib.h>
 #include <string.h>
 
+#ifdef WIN32
+#define __PRETTY_FUNCTION__ __FUNCSIG__
+#endif
+
 /* Shim structs and functions that delegate to libhdfspp and libhdfs. */
 struct hdfs_internal {
   libhdfs_hdfsFS libhdfsRep;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/logging_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/logging_test.cc
index d487bf5f0f0..b14c22b02b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/logging_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/logging_test.cc
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#include <common/logging.h>
-#include <bindings/c/hdfs.cc>
+#include "bindings/c/hdfs.cc"
+#include "common/logging.h"
 
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.cc
index 37fabf568d2..c99087011bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.cc
@@ -18,6 +18,8 @@
 
 #include "mock_connection.h"
 
+#include <boost/asio/io_service.hpp>
+
 namespace hdfs {
 
 MockConnectionBase::MockConnectionBase(boost::asio::io_service *io_service)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h
index 7a7b5f076ed..eb62a2b35c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/mock_connection.h
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 #ifndef LIBHDFSPP_TEST_MOCK_CONNECTION_H_
 #define LIBHDFSPP_TEST_MOCK_CONNECTION_H_
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc
index ccec5812f61..0c22e89f4c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/remote_block_reader_test.cc
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
+#include "reader/block_reader.h"
+#include "reader/datatransfer.h"
+
 #include "mock_connection.h"
 
-#include "datatransfer.pb.h"
 #include "common/util.h"
 #include "common/cancel_tracker.h"
-#include "reader/block_reader.h"
-#include "reader/datatransfer.h"
 #include "reader/fileinfo.h"
 
 #include <google/protobuf/io/coded_stream.h>
@@ -124,11 +124,11 @@ static inline string ToDelimitedString(const pb::MessageLite *msg) {
   return res;
 }
 
-static inline std::pair<error_code, string> Produce(const std::string &s) {
-  return make_pair(error_code(), s);
+static inline std::pair<boost::system::error_code, string> Produce(const std::string &s) {
+  return make_pair(boost::system::error_code(), s);
 }
 
-static inline std::pair<error_code, string> ProducePacket(
+static inline std::pair<boost::system::error_code, string> ProducePacket(
     const std::string &data, const std::string &checksum, int offset_in_block,
     int seqno, bool last_packet) {
   PacketHeaderProto proto;
@@ -148,7 +148,7 @@ static inline std::pair<error_code, string> ProducePacket(
   proto.AppendToString(&payload);
   payload += checksum;
   payload += data;
-  return std::make_pair(error_code(), std::move(payload));
+  return std::make_pair(boost::system::error_code(), std::move(payload));
 }
 
 TEST(RemoteBlockReaderTest, TestReadSingleTrunk) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc
index caf4842b298..ba09e8fc907 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/rpc_engine_test.cc
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-#include "hdfspp/ioservice.h"
+#include "RpcHeader.pb.h"
 
+#include "hdfspp/ioservice.h"
 #include "mock_connection.h"
 #include "test.pb.h"
-#include "RpcHeader.pb.h"
 #include "rpc/rpc_connection_impl.h"
 #include "common/namenode_info.h"
 
@@ -41,6 +41,8 @@ using ::hadoop::common::EmptyResponseProto;
 using ::hadoop::common::EchoRequestProto;
 using ::hadoop::common::EchoResponseProto;
 
+using boost::system::error_code;
+
 using ::testing::Return;
 
 using ::std::make_pair;


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