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 in...@apache.org on 2021/05/26 21:13:19 UTC

[hadoop] branch trunk updated: Revert "Revert "HDFS-15971. Make mkstemp cross platform (#2898)"" (#3044)

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

inigoiri 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 bcaeb1a  Revert "Revert "HDFS-15971. Make mkstemp cross platform (#2898)"" (#3044)
bcaeb1a is described below

commit bcaeb1ac8cd67a14bd332d62bdec1a90ce1c9549
Author: Gautham B A <ga...@gmail.com>
AuthorDate: Thu May 27 02:42:53 2021 +0530

    Revert "Revert "HDFS-15971. Make mkstemp cross platform (#2898)"" (#3044)
---
 .../hadoop-hdfs-native-client/src/CMakeLists.txt   |  3 +-
 .../libhdfs-tests/test_libhdfs_mini_stress.c       |  4 +-
 .../libhdfspp/lib/x-platform/c-api/syscall.cc      | 22 +++++++-
 .../libhdfspp/lib/x-platform/c-api/syscall.h       |  3 +
 .../main/native/libhdfspp/lib/x-platform/syscall.h | 29 ++++++++++
 .../libhdfspp/lib/x-platform/syscall_linux.cc      | 11 ++++
 .../libhdfspp/lib/x-platform/syscall_windows.cc    | 30 ++++++++++
 .../src/main/native/libhdfspp/tests/CMakeLists.txt | 13 +++--
 .../native/libhdfspp/tests/configuration_test.cc   | 23 ++++----
 .../native/libhdfspp/tests/configuration_test.h    | 64 +++++++++++++++++-----
 .../native/libhdfspp/tests/hdfs_builder_test.cc    |  4 +-
 .../libhdfspp/tests/hdfs_configuration_test.cc     | 16 +++---
 .../tests/x-platform/syscall_common_test.cc        | 18 ++++++
 13 files changed, 199 insertions(+), 41 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
index 7c076bd..d8c2012 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
@@ -91,7 +91,8 @@ function(build_libhdfs_test NAME LIBRARY)
             list(APPEND FILES ${CMAKE_SOURCE_DIR}/main/native/libhdfs-tests/${FIL})
         endif()
     endforeach()
-    add_executable("${NAME}_${LIBRARY}" ${FILES})
+    add_executable("${NAME}_${LIBRARY}" $<TARGET_OBJECTS:x_platform_obj_c_api> $<TARGET_OBJECTS:x_platform_obj> ${FILES})
+    target_include_directories("${NAME}_${LIBRARY}" PRIVATE main/native/libhdfspp/lib)
 endfunction()
 
 function(add_libhdfs_test NAME LIBRARY)
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 846852b..1641470 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
@@ -22,6 +22,7 @@
 #include "hdfspp/hdfs_ext.h"
 #include "native_mini_dfs.h"
 #include "os/thread.h"
+#include "x-platform/c-api/syscall.h"
 
 #include <errno.h>
 #include <inttypes.h>
@@ -126,7 +127,8 @@ static int hdfsCurlData(const char *host, const tPort port, const char *dirNm,
   EXPECT_NONNULL(pw = getpwuid(uid));
 
   int fd = -1;
-  EXPECT_NONNEGATIVE(fd = mkstemp(tmpFile));
+  EXPECT_NONNEGATIVE(fd = x_platform_syscall_create_and_open_temp_file(
+                         tmpFile, sizeof tmpFile));
 
   tSize sz = 0;
   while (sz < fileSz) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.cc
index cca9f6a..0bb5fc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.cc
@@ -18,6 +18,26 @@
 
 #include "x-platform/syscall.h"
 
-extern "C" int x_platform_syscall_write_to_stdout(const char* msg) {
+#include <algorithm>
+#include <vector>
+
+extern "C" {
+int x_platform_syscall_write_to_stdout(const char* msg) {
   return XPlatform::Syscall::WriteToStdout(msg) ? 1 : 0;
 }
+
+int x_platform_syscall_create_and_open_temp_file(char* pattern,
+                                                 const size_t pattern_len) {
+  std::vector<char> pattern_vec(pattern, pattern + pattern_len);
+
+  const auto fd = XPlatform::Syscall::CreateAndOpenTempFile(pattern_vec);
+  if (fd != -1) {
+    std::copy_n(pattern_vec.begin(), pattern_len, pattern);
+  }
+  return fd;
+}
+
+int x_platform_syscall_close_file(const int fd) {
+  return XPlatform::Syscall::CloseFile(fd);
+}
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.h
index be905ae..93878b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/syscall.h
@@ -24,5 +24,8 @@
  */
 
 int x_platform_syscall_write_to_stdout(const char* msg);
+int x_platform_syscall_create_and_open_temp_file(char* pattern,
+                                                 size_t pattern_len);
+int x_platform_syscall_close_file(int fd);
 
 #endif  // NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_SYSCALL_H
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall.h
index 3a77f2d..9959f21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall.h
@@ -20,6 +20,7 @@
 #define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_SYSCALL
 
 #include <string>
+#include <vector>
 
 /**
  * The {@link XPlatform} namespace contains components that
@@ -84,6 +85,34 @@ class Syscall {
   static bool StringCompareIgnoreCase(const std::string& a,
                                       const std::string& b);
 
+  /**
+   * Creates and opens a temporary file with a given {@link pattern}.
+   * The {@link pattern} must end with a minimum of 6 'X' characters.
+   * This function will first modify the last 6 'X' characters with
+   * random character values, which serve as the temporary file name.
+   * Subsequently opens the file and returns the file descriptor for
+   * the same. The behaviour of this function is the same as that of
+   * POSIX mkstemp function. The file must be later closed by the
+   * application and is not handled by this function.
+   *
+   * @param pattern the pattern to be used for the temporary filename.
+   * @returns an integer representing the file descriptor for the
+   * opened temporary file. Returns -1 in the case of error and sets
+   * the global errno with the appropriate error code.
+   */
+  static int CreateAndOpenTempFile(std::vector<char>& pattern);
+
+  /**
+   * Closes the file corresponding to given {@link file_descriptor}.
+   *
+   * @param file_descriptor the file descriptor of the file to close.
+   * @returns a boolean indicating the status of the call to this
+   * function. true if it's a success, false in the case of an error.
+   * The global errno is set if the call to this function was not
+   * successful.
+   */
+  static bool CloseFile(int file_descriptor);
+
  private:
   static bool WriteToStdoutImpl(const char* message);
 };
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_linux.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_linux.cc
index 59d93c4..ff02d2f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_linux.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_linux.cc
@@ -21,6 +21,7 @@
 #include <unistd.h>
 
 #include <cstring>
+#include <vector>
 
 #include "syscall.h"
 
@@ -59,3 +60,13 @@ bool XPlatform::Syscall::StringCompareIgnoreCase(const std::string& a,
                                                  const std::string& b) {
   return strcasecmp(a.c_str(), b.c_str()) == 0;
 }
+
+int XPlatform::Syscall::CreateAndOpenTempFile(std::vector<char>& pattern) {
+  // Make space for mkstemp to add NULL character at the end
+  pattern.resize(pattern.size() + 1);
+  return mkstemp(pattern.data());
+}
+
+bool XPlatform::Syscall::CloseFile(const int file_descriptor) {
+  return close(file_descriptor) == 0;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_windows.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_windows.cc
index 2cd9e9d..b5ddd04 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_windows.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/syscall_windows.cc
@@ -19,7 +19,14 @@
 #include <Shlwapi.h>
 #include <WinBase.h>
 #include <Windows.h>
+#include <fcntl.h>
+#include <io.h>
+#include <share.h>
+#include <sys/stat.h>
+#include <sys/types.h>
 
+#include <cerrno>
+#include <cstdlib>
 #include <cstring>
 
 #include "syscall.h"
@@ -64,3 +71,26 @@ bool XPlatform::Syscall::StringCompareIgnoreCase(const std::string& a,
                                                  const std::string& b) {
   return _stricmp(a.c_str(), b.c_str()) == 0;
 }
+
+int XPlatform::Syscall::CreateAndOpenTempFile(std::vector<char>& pattern) {
+  if (_set_errno(0) != 0) {
+    return -1;
+  }
+
+  // Make space for _mktemp_s to add NULL character at the end
+  pattern.resize(pattern.size() + 1);
+  if (_mktemp_s(pattern.data(), pattern.size()) != 0) {
+    return -1;
+  }
+
+  auto fd{-1};
+  if (_sopen_s(&fd, pattern.data(), _O_RDWR | _O_CREAT | _O_EXCL, _SH_DENYNO,
+               _S_IREAD | _S_IWRITE) != 0) {
+    return -1;
+  }
+  return fd;
+}
+
+bool XPlatform::Syscall::CloseFile(const int file_descriptor) {
+  return _close(file_descriptor) == 0;
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
index 32f75f4..d37fcc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt
@@ -96,11 +96,13 @@ add_executable(node_exclusion_test node_exclusion_test.cc)
 target_link_libraries(node_exclusion_test fs gmock_main common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(node_exclusion node_exclusion_test)
 
-add_executable(configuration_test configuration_test.cc)
+add_executable(configuration_test $<TARGET_OBJECTS:x_platform_obj> configuration_test.cc)
+target_include_directories(configuration_test PRIVATE ../lib)
 target_link_libraries(configuration_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(configuration configuration_test)
 
-add_executable(hdfs_configuration_test hdfs_configuration_test.cc)
+add_executable(hdfs_configuration_test $<TARGET_OBJECTS:x_platform_obj> hdfs_configuration_test.cc)
+target_include_directories(hdfs_configuration_test PRIVATE ../lib)
 target_link_libraries(hdfs_configuration_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(hdfs_configuration hdfs_configuration_test)
 
@@ -108,11 +110,13 @@ add_executable(hdfspp_errors_test hdfspp_errors.cc)
 target_link_libraries(hdfspp_errors_test common gmock_main bindings_c fs rpc proto common reader connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(hdfspp_errors hdfspp_errors_test)
 
-add_executable(hdfs_builder_test hdfs_builder_test.cc)
+add_executable(hdfs_builder_test $<TARGET_OBJECTS:x_platform_obj> hdfs_builder_test.cc)
+target_include_directories(hdfs_builder_test PRIVATE ../lib)
 target_link_libraries(hdfs_builder_test test_common gmock_main bindings_c fs rpc proto common reader connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(hdfs_builder_test hdfs_builder_test)
 
 add_executable(logging_test logging_test.cc $<TARGET_OBJECTS:x_platform_obj>)
+target_include_directories(logging_test PRIVATE ../lib)
 target_link_libraries(logging_test common gmock_main bindings_c fs rpc proto common reader connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} gmock_main ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(logging_test logging_test)
 
@@ -124,7 +128,8 @@ add_executable(user_lock_test user_lock_test.cc)
 target_link_libraries(user_lock_test fs gmock_main common ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(user_lock user_lock_test)
 
-add_executable(hdfs_config_connect_bugs_test hdfs_config_connect_bugs.cc)
+add_executable(hdfs_config_connect_bugs_test $<TARGET_OBJECTS:x_platform_obj> hdfs_config_connect_bugs.cc)
+target_include_directories(hdfs_config_connect_bugs_test PRIVATE ../lib)
 target_link_libraries(hdfs_config_connect_bugs_test common gmock_main bindings_c fs rpc proto common reader connection ${PROTOBUF_LIBRARIES} ${OPENSSL_LIBRARIES} ${SASL_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT})
 add_memcheck_test(hdfs_config_connect_bugs hdfs_config_connect_bugs_test)
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc
index 9534204..3bf2524 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc
@@ -299,11 +299,12 @@ TEST(ConfigurationTest, TestFileReads)
   // Single stream
   {
     TempFile tempFile;
-    writeSimpleConfig(tempFile.filename, "key1", "value1");
+    writeSimpleConfig(tempFile.GetFileName(), "key1", "value1");
 
     ConfigurationLoader config_loader;
     config_loader.ClearSearchPath();
-    optional<Configuration> config = config_loader.LoadFromFile<Configuration>(tempFile.filename);
+    optional<Configuration> config =
+        config_loader.LoadFromFile<Configuration>(tempFile.GetFileName());
     EXPECT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
   }
@@ -311,16 +312,18 @@ TEST(ConfigurationTest, TestFileReads)
   // Multiple files
   {
     TempFile tempFile;
-    writeSimpleConfig(tempFile.filename, "key1", "value1");
+    writeSimpleConfig(tempFile.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;
-    optional<Configuration> config = loader.LoadFromFile<Configuration>(tempFile.filename);
+    optional<Configuration> config =
+        loader.LoadFromFile<Configuration>(tempFile.GetFileName());
     ASSERT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
     TempFile tempFile2;
-    writeSimpleConfig(tempFile2.filename, "key2", "value2");
-    optional<Configuration> config2 = loader.OverlayResourceFile(*config, tempFile2.filename);
+    writeSimpleConfig(tempFile2.GetFileName(), "key2", "value2");
+    optional<Configuration> config2 =
+        loader.OverlayResourceFile(*config, tempFile2.GetFileName());
     ASSERT_TRUE(config2 && "Parse second stream");
     EXPECT_EQ("value1", config2->GetWithDefault("key1", ""));
     EXPECT_EQ("value2", config2->GetWithDefault("key2", ""));
@@ -350,13 +353,13 @@ TEST(ConfigurationTest, TestFileReads)
   {
     TempDir tempDir1;
     TempFile tempFile1(tempDir1.path + "/file1.xml");
-    writeSimpleConfig(tempFile1.filename, "key1", "value1");
+    writeSimpleConfig(tempFile1.GetFileName(), "key1", "value1");
     TempDir tempDir2;
     TempFile tempFile2(tempDir2.path + "/file2.xml");
-    writeSimpleConfig(tempFile2.filename, "key2", "value2");
+    writeSimpleConfig(tempFile2.GetFileName(), "key2", "value2");
     TempDir tempDir3;
     TempFile tempFile3(tempDir3.path + "/file3.xml");
-    writeSimpleConfig(tempFile3.filename, "key3", "value3");
+    writeSimpleConfig(tempFile3.GetFileName(), "key3", "value3");
 
     ConfigurationLoader loader;
     loader.SetSearchPath(tempDir1.path + ":" + tempDir2.path + ":" + tempDir3.path);
@@ -377,7 +380,7 @@ TEST(ConfigurationTest, TestDefaultConfigs) {
   {
     TempDir tempDir;
     TempFile coreSite(tempDir.path + "/core-site.xml");
-    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;
     loader.SetSearchPath(tempDir.path);
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h
index 9ad11b7..23fc0d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h
@@ -21,11 +21,19 @@
 #include "hdfspp/config_parser.h"
 #include "common/configuration.h"
 #include "common/configuration_loader.h"
+#include "x-platform/syscall.h"
+
 #include <cstdio>
 #include <fstream>
 #include <istream>
+#include <string>
+#include <utility>
+#include <vector>
+
 #include <ftw.h>
+#include <unistd.h>
 #include <gmock/gmock.h>
+#include <gtest/gtest.h>
 
 namespace hdfs {
 
@@ -107,23 +115,51 @@ void writeDamagedConfig(const std::string& filename, Args... args) {
 
 // TempDir: is deleted on destruction
 class TempFile {
-public:
-  std::string filename;
-  char        fn_buffer[128];
-  int         tempFileHandle;
-  TempFile() : tempFileHandle(-1) {
-    strncpy(fn_buffer, "/tmp/test_XXXXXXXXXX", sizeof(fn_buffer));
-    tempFileHandle = mkstemp(fn_buffer);
-    EXPECT_NE(-1, tempFileHandle);
-    filename = fn_buffer;
+ public:
+  TempFile() {
+    std::vector<char> tmp_buf(filename_.begin(), filename_.end());
+    fd_ = XPlatform::Syscall::CreateAndOpenTempFile(tmp_buf);
+    EXPECT_NE(fd_, -1);
+    filename_.assign(tmp_buf.data());
   }
-  TempFile(const std::string & fn) : filename(fn), tempFileHandle(-1) {
-    strncpy(fn_buffer, fn.c_str(), sizeof(fn_buffer));
-    fn_buffer[sizeof(fn_buffer)-1] = 0;
+
+  TempFile(std::string fn) : filename_(std::move(fn)) {}
+
+  TempFile(const TempFile& other) = default;
+
+  TempFile(TempFile&& other) noexcept
+      : filename_{std::move(other.filename_)}, fd_{other.fd_} {}
+
+  TempFile& operator=(const TempFile& other) {
+    if (&other != this) {
+      filename_ = other.filename_;
+      fd_ = other.fd_;
+    }
+    return *this;
   }
-  ~TempFile() { if(-1 != tempFileHandle) close(tempFileHandle); unlink(fn_buffer); }
-};
 
+  TempFile& operator=(TempFile&& other) noexcept {
+    if (&other != this) {
+      filename_ = std::move(other.filename_);
+      fd_ = other.fd_;
+    }
+    return *this;
+  }
+
+  [[nodiscard]] const std::string& GetFileName() const { return filename_; }
+
+  ~TempFile() {
+    if (-1 != fd_) {
+      EXPECT_NE(XPlatform::Syscall::CloseFile(fd_), -1);
+    }
+
+    unlink(filename_.c_str());
+  }
+
+ private:
+  std::string filename_{"/tmp/test_XXXXXXXXXX"};
+  int fd_{-1};
+};
 
 // Callback to remove a directory in the nftw visitor
 int nftw_remove(const char *fpath, const struct stat *sb, int typeflag, struct FTW *ftwbuf)
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_builder_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_builder_test.cc
index 01db69d..147cfee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_builder_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_builder_test.cc
@@ -45,7 +45,7 @@ TEST(HdfsBuilderTest, TestRead)
   {
     TempDir tempDir1;
     TempFile tempFile1(tempDir1.path + "/core-site.xml");
-    writeSimpleConfig(tempFile1.filename, "key1", "value1");
+    writeSimpleConfig(tempFile1.GetFileName(), "key1", "value1");
 
     hdfsBuilder * builder = hdfsNewBuilderFromDirectory(tempDir1.path.c_str());
 
@@ -68,7 +68,7 @@ TEST(HdfsBuilderTest, TestRead)
   {
     TempDir tempDir1;
     TempFile tempFile1(tempDir1.path + "/core-site.xml");
-    writeSimpleConfig(tempFile1.filename, "key1", "100");
+    writeSimpleConfig(tempFile1.GetFileName(), "key1", "100");
 
     hdfsBuilder * builder = hdfsNewBuilderFromDirectory(tempDir1.path.c_str());
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc
index b21725c..4e1bc3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc
@@ -72,9 +72,9 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
   {
     TempDir tempDir;
     TempFile coreSite(tempDir.path + "/core-site.xml");
-    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
     TempFile hdfsSite(tempDir.path + "/hdfs-site.xml");
-    writeSimpleConfig(hdfsSite.filename, "key2", "value2");
+    writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigurationLoader loader;
     loader.SetSearchPath(tempDir.path);
@@ -89,7 +89,7 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
   {
     TempDir tempDir;
     TempFile coreSite(tempDir.path + "/core-site.xml");
-    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;
     loader.SetSearchPath(tempDir.path);
@@ -103,7 +103,7 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
   {
     TempDir tempDir;
     TempFile hdfsSite(tempDir.path + "/hdfs-site.xml");
-    writeSimpleConfig(hdfsSite.filename, "key2", "value2");
+    writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigurationLoader loader;
     loader.SetSearchPath(tempDir.path);
@@ -121,9 +121,9 @@ TEST(HdfsConfigurationTest, TestConfigParserAPI) {
   {
     TempDir tempDir;
     TempFile coreSite(tempDir.path + "/core-site.xml");
-    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
     TempFile hdfsSite(tempDir.path + "/hdfs-site.xml");
-    writeSimpleConfig(hdfsSite.filename, "key2", "value2");
+    writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigParser parser(tempDir.path);
 
@@ -142,9 +142,9 @@ TEST(HdfsConfigurationTest, TestConfigParserAPI) {
   {
     TempDir tempDir;
     TempFile coreSite(tempDir.path + "/core-site.xml");
-    writeSimpleConfig(coreSite.filename, "key1", "value1");
+    writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
     TempFile hdfsSite(tempDir.path + "/hdfs-site.xml");
-    writeDamagedConfig(hdfsSite.filename, "key2", "value2");
+    writeDamagedConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigParser parser(tempDir.path);
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/syscall_common_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/syscall_common_test.cc
index 7fa3971..a7847e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/syscall_common_test.cc
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/syscall_common_test.cc
@@ -85,3 +85,21 @@ TEST(XPlatformSyscall, StringCompareIgnoreCaseNegative) {
   EXPECT_FALSE(XPlatform::Syscall::StringCompareIgnoreCase("abcd", "abcde"));
   EXPECT_FALSE(XPlatform::Syscall::StringCompareIgnoreCase("12345", "abcde"));
 }
+
+TEST(XPlatformSyscall, CreateAndOpenTempFileBasic) {
+  std::string pattern("tmp-XXXXXX");
+  std::vector<char> pattern_vec(pattern.begin(), pattern.end());
+
+  const auto fd = XPlatform::Syscall::CreateAndOpenTempFile(pattern_vec);
+  EXPECT_GT(fd, -1);
+  EXPECT_TRUE(XPlatform::Syscall::CloseFile(fd));
+}
+
+TEST(XPlatformSyscall, CreateAndOpenTempFileNegative) {
+  std::string pattern("does-not-adhere-to-pattern");
+  std::vector<char> pattern_vec(pattern.begin(), pattern.end());
+
+  const auto fd = XPlatform::Syscall::CreateAndOpenTempFile(pattern_vec);
+  EXPECT_EQ(fd, -1);
+  EXPECT_FALSE(XPlatform::Syscall::CloseFile(fd));
+}

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