Browse Source

HDFS-16174. Refactor TempFile and TempDir in libhdfs++ (#3303)

Gautham B A 3 years ago
parent
commit
409878ef80

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt

@@ -66,6 +66,7 @@ endfunction(add_memcheck_test)
 #
 
 add_subdirectory(x-platform)
+add_subdirectory(utils)
 
 add_executable(uri_test uri_test.cc)
 target_link_libraries(uri_test common gmock_main ${CMAKE_THREAD_LIBS_INIT})
@@ -96,12 +97,12 @@ 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 $<TARGET_OBJECTS:x_platform_obj> configuration_test.cc)
+add_executable(configuration_test $<TARGET_OBJECTS:test_utils> 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 $<TARGET_OBJECTS:x_platform_obj> hdfs_configuration_test.cc)
+add_executable(hdfs_configuration_test $<TARGET_OBJECTS:test_utils> 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)
@@ -110,7 +111,7 @@ 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 $<TARGET_OBJECTS:x_platform_obj> hdfs_builder_test.cc)
+add_executable(hdfs_builder_test $<TARGET_OBJECTS:test_utils> 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)
@@ -128,7 +129,7 @@ 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 $<TARGET_OBJECTS:x_platform_obj> hdfs_config_connect_bugs.cc)
+add_executable(hdfs_config_connect_bugs_test $<TARGET_OBJECTS:test_utils> 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)

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.cc

@@ -19,6 +19,9 @@
 #include "configuration_test.h"
 #include "common/configuration.h"
 #include "common/configuration_loader.h"
+#include "utils/temp-file.h"
+#include "utils/temp-dir.h"
+
 #include <gmock/gmock.h>
 #include <cstdio>
 #include <fstream>
@@ -298,7 +301,7 @@ TEST(ConfigurationTest, TestFileReads)
 {
   // Single stream
   {
-    TempFile tempFile;
+    TestUtils::TempFile tempFile;
     writeSimpleConfig(tempFile.GetFileName(), "key1", "value1");
 
     ConfigurationLoader config_loader;
@@ -311,7 +314,7 @@ TEST(ConfigurationTest, TestFileReads)
 
   // Multiple files
   {
-    TempFile tempFile;
+    TestUtils::TempFile tempFile;
     writeSimpleConfig(tempFile.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;
@@ -320,7 +323,7 @@ TEST(ConfigurationTest, TestFileReads)
     ASSERT_TRUE(config && "Parse first stream");
     EXPECT_EQ("value1", config->GetWithDefault("key1", ""));
 
-    TempFile tempFile2;
+    TestUtils::TempFile tempFile2;
     writeSimpleConfig(tempFile2.GetFileName(), "key2", "value2");
     optional<Configuration> config2 =
         loader.OverlayResourceFile(*config, tempFile2.GetFileName());
@@ -331,7 +334,7 @@ TEST(ConfigurationTest, TestFileReads)
 
   // Try to add a directory
   {
-    TempDir tempDir;
+    TestUtils::TempDir tempDir;
 
     ConfigurationLoader config_loader;
     config_loader.ClearSearchPath();
@@ -351,14 +354,14 @@ TEST(ConfigurationTest, TestFileReads)
 
   // Search path
   {
-    TempDir tempDir1;
-    TempFile tempFile1(tempDir1.GetPath() + "/file1.xml");
+    TestUtils::TempDir tempDir1;
+    TestUtils::TempFile tempFile1(tempDir1.GetPath() + "/file1.xml");
     writeSimpleConfig(tempFile1.GetFileName(), "key1", "value1");
-    TempDir tempDir2;
-    TempFile tempFile2(tempDir2.GetPath() + "/file2.xml");
+    TestUtils::TempDir tempDir2;
+    TestUtils::TempFile tempFile2(tempDir2.GetPath() + "/file2.xml");
     writeSimpleConfig(tempFile2.GetFileName(), "key2", "value2");
-    TempDir tempDir3;
-    TempFile tempFile3(tempDir3.GetPath() + "/file3.xml");
+    TestUtils::TempDir tempDir3;
+    TestUtils::TempFile tempFile3(tempDir3.GetPath() + "/file3.xml");
     writeSimpleConfig(tempFile3.GetFileName(), "key3", "value3");
 
     ConfigurationLoader loader;
@@ -379,8 +382,8 @@ TEST(ConfigurationTest, TestFileReads)
 TEST(ConfigurationTest, TestDefaultConfigs) {
   // Search path
   {
-    TempDir tempDir;
-    TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
     writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;

+ 0 - 102
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/configuration_test.h

@@ -23,7 +23,6 @@
 #include "hdfspp/config_parser.h"
 #include "common/configuration.h"
 #include "common/configuration_loader.h"
-#include "x-platform/syscall.h"
 
 #include <cstdio>
 #include <fstream>
@@ -32,8 +31,6 @@
 #include <utility>
 #include <vector>
 
-#include <ftw.h>
-#include <unistd.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
 
@@ -114,105 +111,6 @@ void writeDamagedConfig(const std::string& filename, Args... args) {
   out.open(filename);
   out << stream.rdbuf();
 }
-
-// TempDir: is deleted on destruction
-class TempFile {
- 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(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& 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)
-{
-  (void)sb; (void)typeflag; (void)ftwbuf;
-
-  int rv = remove(fpath);
-  EXPECT_EQ(0, rv);
-  return rv;
-}
-
-// TempDir: is created in ctor and recursively deletes in dtor
-class TempDir {
- public:
-  TempDir() {
-    std::vector<char> path_pattern(path_.begin(), path_.end());
-    is_path_init_ = XPlatform::Syscall::CreateTempDir(path_pattern);
-    EXPECT_TRUE(is_path_init_);
-    path_.assign(path_pattern.data());
-  }
-
-  TempDir(const TempDir& other) = default;
-
-  TempDir(TempDir&& other) noexcept : path_{std::move(other.path_)} {}
-
-  TempDir& operator=(const TempDir& other) {
-    if (&other != this) {
-      path_ = other.path_;
-    }
-    return *this;
-  }
-
-  TempDir& operator=(TempDir&& other) noexcept {
-    if (&other != this) {
-      path_ = std::move(other.path_);
-    }
-    return *this;
-  }
-
-  [[nodiscard]] const std::string& GetPath() const { return path_; }
-
-  ~TempDir() {
-    if (is_path_init_) {
-      nftw(path_.c_str(), nftw_remove, 64, FTW_DEPTH | FTW_PHYS);
-    }
-  }
-
- private:
-  std::string path_{"/tmp/test_dir_XXXXXXXXXX"};
-  bool is_path_init_{false};
-};
 }
 
 #endif

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_builder_test.cc

@@ -18,6 +18,9 @@
 
 #include "hdfspp/hdfs_ext.h"
 #include "configuration_test.h"
+#include "utils/temp-file.h"
+#include "utils/temp-dir.h"
+
 #include <gmock/gmock.h>
 #include <google/protobuf/stubs/common.h>
 
@@ -27,7 +30,7 @@ using namespace hdfs;
 
 TEST(HdfsBuilderTest, TestStubBuilder) {
   {
-    TempDir tempDir1;
+    TestUtils::TempDir tempDir1;
 
     hdfsBuilder *builder =
         hdfsNewBuilderFromDirectory(tempDir1.GetPath().c_str());
@@ -44,8 +47,8 @@ TEST(HdfsBuilderTest, TestRead)
 {
   // Reading string values
   {
-    TempDir tempDir1;
-    TempFile tempFile1(tempDir1.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir1;
+    TestUtils::TempFile tempFile1(tempDir1.GetPath() + "/core-site.xml");
     writeSimpleConfig(tempFile1.GetFileName(), "key1", "value1");
 
     hdfsBuilder *builder =
@@ -68,8 +71,8 @@ TEST(HdfsBuilderTest, TestRead)
 
   // Reading int values
   {
-    TempDir tempDir1;
-    TempFile tempFile1(tempDir1.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir1;
+    TestUtils::TempFile tempFile1(tempDir1.GetPath() + "/core-site.xml");
     writeSimpleConfig(tempFile1.GetFileName(), "key1", "100");
 
     hdfsBuilder *builder =

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_config_connect_bugs.cc

@@ -27,6 +27,8 @@
 #include <chrono>
 #include <exception>
 
+#include "utils/temp-dir.h"
+
 static const char *hdfs_11294_core_site_txt =
 "<configuration>\n"
 "  <property name=\"fs.defaultFS\" value=\"hdfs://NAMESERVICE1\"/>\n"
@@ -78,7 +80,7 @@ namespace hdfs {
 // Make sure we can set up a mini-cluster and connect to it
 TEST(ConfigConnectBugs, Test_HDFS_11294) {
   // Directory for hdfs config
-  TempDir td;
+  TestUtils::TempDir td;
 
   const std::string &tempDirPath = td.GetPath();
   const std::string coreSitePath = tempDirPath + "/core-site.xml";

+ 16 - 13
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_configuration_test.cc

@@ -18,6 +18,9 @@
 
 #include "common/hdfs_configuration.h"
 #include "configuration_test.h"
+#include "utils/temp-file.h"
+#include "utils/temp-dir.h"
+
 #include <gmock/gmock.h>
 #include <iostream>
 
@@ -70,10 +73,10 @@ TEST(HdfsConfigurationTest, TestSetOptions)
 TEST(HdfsConfigurationTest, TestDefaultConfigs) {
   // Search path
   {
-    TempDir tempDir;
-    TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
     writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
-    TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
+    TestUtils::TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
     writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigurationLoader loader;
@@ -87,8 +90,8 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
 
   // Only core-site.xml available
   {
-    TempDir tempDir;
-    TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
     writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
 
     ConfigurationLoader loader;
@@ -101,8 +104,8 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
 
   // Only hdfs-site available
   {
-    TempDir tempDir;
-    TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
     writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigurationLoader loader;
@@ -119,10 +122,10 @@ TEST(HdfsConfigurationTest, TestDefaultConfigs) {
 TEST(HdfsConfigurationTest, TestConfigParserAPI) {
   // Config parser API
   {
-    TempDir tempDir;
-    TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
     writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
-    TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
+    TestUtils::TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
     writeSimpleConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigParser parser(tempDir.GetPath());
@@ -140,10 +143,10 @@ TEST(HdfsConfigurationTest, TestConfigParserAPI) {
   }
 
   {
-    TempDir tempDir;
-    TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
+    TestUtils::TempDir tempDir;
+    TestUtils::TempFile coreSite(tempDir.GetPath() + "/core-site.xml");
     writeSimpleConfig(coreSite.GetFileName(), "key1", "value1");
-    TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
+    TestUtils::TempFile hdfsSite(tempDir.GetPath() + "/hdfs-site.xml");
     writeDamagedConfig(hdfsSite.GetFileName(), "key2", "value2");
 
     ConfigParser parser(tempDir.GetPath());

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/utils/CMakeLists.txt

@@ -0,0 +1,19 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+add_library(test_utils OBJECT $<TARGET_OBJECTS:x_platform_obj> temp-file.cc temp-dir.cc)

+ 75 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/utils/temp-dir.cc

@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+#include <vector>
+
+#include <ftw.h>
+#include <gtest/gtest.h>
+#include <sys/stat.h>
+
+#include "utils/temp-dir.h"
+#include "x-platform/syscall.h"
+
+namespace TestUtils {
+/*
+ * Callback to remove a directory in the nftw visitor.
+ */
+int nftw_remove(const char *fpath, const struct stat *sb, int typeflag,
+                struct FTW *ftwbuf);
+
+TempDir::TempDir() {
+  std::vector path_pattern(path_.begin(), path_.end());
+  is_path_init_ = XPlatform::Syscall::CreateTempDir(path_pattern);
+  EXPECT_TRUE(is_path_init_);
+  path_.assign(path_pattern.data());
+}
+
+TempDir::TempDir(TempDir &&other) noexcept : path_{std::move(other.path_)} {}
+
+TempDir &TempDir::operator=(const TempDir &other) {
+  if (&other != this) {
+    path_ = other.path_;
+  }
+  return *this;
+}
+
+TempDir &TempDir::operator=(TempDir &&other) noexcept {
+  if (&other != this) {
+    path_ = std::move(other.path_);
+  }
+  return *this;
+}
+
+TempDir::~TempDir() {
+  if (is_path_init_) {
+    nftw(path_.c_str(), nftw_remove, 64, FTW_DEPTH | FTW_PHYS);
+  }
+}
+
+int nftw_remove(const char *fpath, const struct stat *sb, int typeflag,
+                FTW *ftwbuf) {
+  (void)sb;
+  (void)typeflag;
+  (void)ftwbuf;
+
+  int rv = remove(fpath);
+  EXPECT_EQ(0, rv);
+  return rv;
+}
+} // namespace TestUtils

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/utils/temp-dir.h

@@ -0,0 +1,53 @@
+/**
+ * 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 NATIVE_LIBHDFSPP_TESTS_UTILS_TEMP_DIR
+#define NATIVE_LIBHDFSPP_TESTS_UTILS_TEMP_DIR
+
+#include <string>
+
+namespace TestUtils {
+/*
+ * Creates a temporary directory and deletes its contents recursively
+ * upon destruction of its instance.
+ *
+ * Creates a directory in /tmp by default.
+ */
+class TempDir {
+public:
+  TempDir();
+
+  TempDir(const TempDir &other) = default;
+
+  TempDir(TempDir &&other) noexcept;
+
+  TempDir &operator=(const TempDir &other);
+
+  TempDir &operator=(TempDir &&other) noexcept;
+
+  [[nodiscard]] const std::string &GetPath() const { return path_; }
+
+  ~TempDir();
+
+private:
+  std::string path_{"/tmp/test_dir_XXXXXXXXXX"};
+  bool is_path_init_{false};
+};
+} // namespace TestUtils
+
+#endif

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/utils/temp-file.cc

@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "utils/temp-file.h"
+#include "x-platform/syscall.h"
+
+namespace TestUtils {
+TempFile::TempFile() {
+  std::vector tmp_buf(filename_.begin(), filename_.end());
+  fd_ = XPlatform::Syscall::CreateAndOpenTempFile(tmp_buf);
+  EXPECT_NE(fd_, -1);
+  filename_.assign(tmp_buf.data());
+}
+
+TempFile::TempFile(std::string filename) : filename_(std::move(filename)) {}
+
+TempFile::TempFile(TempFile &&other) noexcept
+    : filename_{std::move(other.filename_)}, fd_{other.fd_} {}
+
+TempFile &TempFile::operator=(const TempFile &other) {
+  if (&other != this) {
+    filename_ = other.filename_;
+    fd_ = other.fd_;
+  }
+  return *this;
+}
+
+TempFile &TempFile::operator=(TempFile &&other) noexcept {
+  if (&other != this) {
+    filename_ = std::move(other.filename_);
+    fd_ = other.fd_;
+  }
+  return *this;
+}
+
+TempFile::~TempFile() {
+  if (-1 != fd_) {
+    EXPECT_NE(XPlatform::Syscall::CloseFile(fd_), -1);
+  }
+
+  unlink(filename_.c_str());
+}
+} // namespace TestUtils

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/utils/temp-file.h

@@ -0,0 +1,56 @@
+/**
+ * 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 NATIVE_LIBHDFSPP_TESTS_UTILS_TEMP_FILE
+#define NATIVE_LIBHDFSPP_TESTS_UTILS_TEMP_FILE
+
+#include <string>
+
+namespace TestUtils {
+/**
+ * Creates a temporary file and deletes it
+ * upon destruction of the TempFile instance.
+ *
+ * The temporary file gets created in /tmp directory
+ * by default.
+ */
+class TempFile {
+public:
+  TempFile();
+
+  TempFile(std::string filename);
+
+  TempFile(const TempFile &other) = default;
+
+  TempFile(TempFile &&other) noexcept;
+
+  TempFile &operator=(const TempFile &other);
+
+  TempFile &operator=(TempFile &&other) noexcept;
+
+  [[nodiscard]] const std::string &GetFileName() const { return filename_; }
+
+  ~TempFile();
+
+private:
+  std::string filename_{"/tmp/test_XXXXXXXXXX"};
+  int fd_{-1};
+};
+} // namespace TestUtils
+
+#endif