Parcourir la source

HDFS-16463. Make dirent cross platform compatible (#4370)

* jnihelper.c in HDFS native client uses
  dirent.h. This header file isn't available
  on Windows.
* This PR provides a cross platform
  compatible implementation for dirent
  under the XPlatform library.
Gautham B A il y a 3 ans
Parent
commit
d557c44d94
15 fichiers modifiés avec 735 ajouts et 3 suppressions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt
  2. 2 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c
  4. 3 2
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt
  5. 68 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h
  6. 92 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc
  7. 39 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h
  8. 35 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h
  9. 38 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc
  10. 77 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h
  11. 12 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt
  12. 100 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc
  13. 37 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h
  14. 151 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc
  15. 78 0
      hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt

@@ -35,6 +35,8 @@ add_library(native_mini_dfs
     ../libhdfs/jni_helper.c
     ${OS_DIR}/mutexes.c
     ${OS_DIR}/thread_local_storage.c
+    $<TARGET_OBJECTS:x_platform_obj>
+    $<TARGET_OBJECTS:x_platform_obj_c_api>
 )
 
 add_executable(test_native_mini_dfs test_native_mini_dfs.c)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt

@@ -39,6 +39,8 @@ hadoop_add_dual_library(hdfs
     jclasses.c
     ${OS_DIR}/mutexes.c
     ${OS_DIR}/thread_local_storage.c
+    $<TARGET_OBJECTS:x_platform_obj>
+    $<TARGET_OBJECTS:x_platform_obj_c_api>
 )
 if(NEED_LINK_DL)
    set(LIB_DL dl)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c

@@ -23,10 +23,10 @@
 #include "platform.h"
 #include "os/mutexes.h"
 #include "os/thread_local_storage.h"
+#include "x-platform/c-api/dirent.h"
 #include "x-platform/types.h"
 
 #include <errno.h>
-#include <dirent.h>
 #include <stdio.h> 
 #include <string.h> 
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt

@@ -22,5 +22,6 @@ else()
   set(SYSCALL_SRC syscall_linux.cc)
 endif()
 
-add_library(x_platform_obj OBJECT ${SYSCALL_SRC} utils.cc)
-add_library(x_platform_obj_c_api OBJECT $<TARGET_OBJECTS:x_platform_obj> c-api/syscall.cc)
+add_library(x_platform_obj OBJECT ${SYSCALL_SRC} utils.cc dirent.cc)
+add_library(x_platform_obj_c_api OBJECT $<TARGET_OBJECTS:x_platform_obj> c-api/syscall.cc c-api/dirent.cc)
+target_compile_definitions(x_platform_obj_c_api PRIVATE USE_X_PLATFORM_DIRENT)

+ 68 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h

@@ -0,0 +1,68 @@
+/**
+ * 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_LIB_CROSS_PLATFORM_C_API_CORE_DIRENT_H
+#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_CORE_DIRENT_H
+
+/**
+ * DIR struct holds the pointer to XPlatform::Dirent instance. Since this will
+ * be used in C, we can't hold the pointer to XPlatform::Dirent. We're working
+ * around this by using a void pointer and casting it to XPlatform::Dirent when
+ * needed in C++.
+ */
+typedef struct DIR {
+  void *x_platform_dirent_ptr;
+} DIR;
+
+/**
+ * dirent struct contains the name of the file/folder while iterating through
+ * the directory's children.
+ */
+struct dirent {
+  char d_name[256];
+};
+
+/**
+ * Opens a directory for iteration. Internally, it instantiates DIR struct for
+ * the given path. closedir must be called on the returned pointer to DIR struct
+ * when done.
+ *
+ * @param dir_path The path to the directory to iterate through.
+ * @return A pointer to the DIR struct.
+ */
+DIR *opendir(const char *dir_path);
+
+/**
+ * For iterating through the children of the directory pointed to by the DIR
+ * struct pointer.
+ *
+ * @param dir The pointer to the DIR struct.
+ * @return A pointer to dirent struct containing the name of the current child
+ * file/folder.
+ */
+struct dirent *readdir(DIR *dir);
+
+/**
+ * De-allocates the XPlatform::Dirent instance pointed to by the DIR pointer.
+ *
+ * @param dir The pointer to DIR struct to close.
+ * @return 0 if successful.
+ */
+int closedir(DIR *dir);
+
+#endif

+ 92 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc

@@ -0,0 +1,92 @@
+/**
+ * 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 <algorithm>
+#include <cerrno>
+#include <iostream>
+#include <iterator>
+#include <system_error>
+#include <variant>
+
+#include "x-platform/c-api/dirent.h"
+#include "x-platform/dirent.h"
+
+DIR *opendir(const char *dir_path) {
+  const auto dir = new DIR;
+  dir->x_platform_dirent_ptr = new XPlatform::Dirent(dir_path);
+  return dir;
+}
+
+struct dirent *readdir(DIR *dir) {
+  /*
+   * We will use a static variable to hold the dirent, so that we align with the
+   * readdir's implementation in dirent.h header file in Linux.
+   */
+  static struct dirent static_dir_entry;
+
+  // Get the XPlatform::Dirent instance and move the iterator.
+  const auto x_platform_dirent =
+      static_cast<XPlatform::Dirent *>(dir->x_platform_dirent_ptr);
+  const auto dir_entry = x_platform_dirent->NextFile();
+
+  // End of iteration.
+  if (std::holds_alternative<std::monostate>(dir_entry)) {
+    return nullptr;
+  }
+
+  // Error in iteration.
+  if (std::holds_alternative<std::error_code>(dir_entry)) {
+    const auto err = std::get<std::error_code>(dir_entry);
+    errno = err.value();
+
+#ifdef X_PLATFORM_C_API_DIRENT_DEBUG
+    std::cerr << "Error in listing directory: " << err.message() << std::endl;
+#endif
+
+    return nullptr;
+  }
+
+  // Return the current child file/folder's name.
+  if (std::holds_alternative<std::filesystem::directory_entry>(dir_entry)) {
+    const auto entry = std::get<std::filesystem::directory_entry>(dir_entry);
+    const auto filename = entry.path().filename().string();
+
+    // The file name's length shouldn't exceed 256.
+    if (filename.length() >= 256) {
+      errno = 1;
+      return nullptr;
+    }
+
+    std::fill(std::begin(static_dir_entry.d_name),
+              std::end(static_dir_entry.d_name), '\0');
+    std::copy(filename.begin(), filename.end(),
+              std::begin(static_dir_entry.d_name));
+  }
+  return &static_dir_entry;
+}
+
+int closedir(DIR *dir) {
+  const auto x_platform_dirent =
+      static_cast<XPlatform::Dirent *>(dir->x_platform_dirent_ptr);
+  delete x_platform_dirent;
+  delete dir;
+
+  // We can't use the void return type for closedir since we want to align the
+  // closedir method's signature in dirent.h header file in Linux.
+  return 0;
+}

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h

@@ -0,0 +1,39 @@
+/**
+ * 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_LIB_CROSS_PLATFORM_C_API_DIRENT_H
+#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_DIRENT_H
+
+#if !(defined(WIN32) || defined(USE_X_PLATFORM_DIRENT))
+
+/*
+ * For non-Windows environments, we use the dirent.h header itself.
+ */
+#include <dirent.h>
+
+#else
+
+/*
+ * If it's a Windows environment or if the macro USE_X_PLATFORM_DIRENT is
+ * defined, we switch to using dirent from the XPlatform library.
+ */
+#include "x-platform/c-api/extern/dirent.h"
+
+#endif
+
+#endif

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h

@@ -0,0 +1,35 @@
+/**
+ * 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_LIB_CROSS_PLATFORM_C_API_EXTERN_DIRENT_H
+#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_EXTERN_DIRENT_H
+
+/*
+ * We will use extern "C" only on Windows.
+ */
+#if defined(WIN32) && defined(__cplusplus)
+extern "C" {
+#endif
+
+#include "x-platform/c-api/core/dirent.h"
+
+#if defined(WIN32) && defined(__cplusplus)
+}
+#endif
+
+#endif

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc

@@ -0,0 +1,38 @@
+/**
+ * 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 <filesystem>
+#include <system_error>
+#include <variant>
+
+#include "dirent.h"
+
+std::variant<std::monostate, std::filesystem::directory_entry, std::error_code>
+XPlatform::Dirent::NextFile() {
+  if (dir_it_err_) {
+    return dir_it_err_;
+  }
+
+  if (dir_it_ == std::filesystem::end(dir_it_)) {
+    return std::monostate();
+  }
+
+  const std::filesystem::directory_entry dir_entry = *dir_it_;
+  dir_it_ = dir_it_.increment(dir_it_err_);
+  return dir_entry;
+}

+ 77 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h

@@ -0,0 +1,77 @@
+/**
+ * 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_LIB_CROSS_PLATFORM_DIRENT
+#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_DIRENT
+
+#include <filesystem>
+#include <string>
+#include <system_error>
+#include <variant>
+
+namespace XPlatform {
+/**
+ * {@class XPlatform::Dirent} provides the functionality to perform a one-time
+ * iteration per {@link XPlatform::Dirent} through the child files or folders
+ * under a given path.
+ */
+class Dirent {
+public:
+  Dirent(const std::string &path)
+      : dir_it_{std::filesystem::path{path}, dir_it_err_} {}
+
+  // Abiding to the Rule of 5
+  Dirent(const Dirent &) = default;
+  Dirent(Dirent &&) = default;
+  Dirent &operator=(const Dirent &) = default;
+  Dirent &operator=(Dirent &&) = default;
+  ~Dirent() = default;
+
+  /**
+   * Advances the iterator {@link XPlatform::Dirent#dir_it_} to the next file in
+   * the given path.
+   *
+   * @return An {@link std::variant} comprising of any one of the following
+   * types:
+   * 1. {@link std::monostate} which indicates the end of iteration of all the
+   * files in the given path.
+   * 2. {@link std::filesystem::directory_entry} which is the directory entry of
+   * the current file.
+   * 3. {@link std::error_code} which corresponds to the error in retrieving the
+   * file.
+   */
+  std::variant<std::monostate, std::filesystem::directory_entry,
+               std::error_code>
+  NextFile();
+
+private:
+  /**
+   * Indicates the error corresponding to the most recent invocation of
+   * directory iteration by {@link XPlatform::Dirent#dir_it_}.
+   */
+  std::error_code dir_it_err_{};
+
+  /**
+   * The iterator used for iterating through the files or folders under the
+   * given path.
+   */
+  std::filesystem::directory_iterator dir_it_;
+};
+} // namespace XPlatform
+
+#endif

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt

@@ -36,3 +36,15 @@ add_executable(x_platform_types_test types_test.cc)
 target_include_directories(x_platform_types_test PRIVATE ${LIBHDFSPP_LIB_DIR})
 target_link_libraries(x_platform_types_test gtest_main)
 add_test(x_platform_types_test x_platform_types_test)
+
+add_library(x_platform_dirent_test_obj OBJECT $<TARGET_OBJECTS:x_platform_obj> dirent_test.cc)
+add_executable(x_platform_dirent_test $<TARGET_OBJECTS:x_platform_dirent_test_obj> $<TARGET_OBJECTS:x_platform_obj>)
+target_include_directories(x_platform_dirent_test PRIVATE ${LIBHDFSPP_LIB_DIR})
+target_link_libraries(x_platform_dirent_test PRIVATE gtest_main)
+add_test(x_platform_dirent_test x_platform_dirent_test)
+
+add_executable(x_platform_dirent_c_test $<TARGET_OBJECTS:x_platform_dirent_test_obj> $<TARGET_OBJECTS:x_platform_obj> $<TARGET_OBJECTS:x_platform_obj_c_api> c-api/dirent_test.cc)
+target_compile_definitions(x_platform_dirent_c_test PRIVATE USE_X_PLATFORM_DIRENT)
+target_include_directories(x_platform_dirent_c_test PRIVATE ${LIBHDFSPP_LIB_DIR} ../)
+target_link_libraries(x_platform_dirent_c_test PRIVATE gtest_main)
+add_test(x_platform_dirent_c_test x_platform_dirent_c_test)

+ 100 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc

@@ -0,0 +1,100 @@
+/**
+ * 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 <cassert>
+#include <cerrno>
+#include <iostream>
+#include <sstream>
+#include <stdexcept>
+#include <unordered_set>
+
+#include "x-platform/c-api/dirent.h"
+#include "x-platform/c-api/dirent_test.h"
+
+std::unordered_set<std::string>
+DirentCApiTest::ListDirAndFiles(const std::string &path) const {
+  std::stringstream err_msg;
+  std::unordered_set<std::string> paths;
+
+  DIR *dir = opendir(path.c_str());
+  if (dir == nullptr) {
+    err_msg << "Unable to open directory " << path;
+    throw std::runtime_error(err_msg.str());
+  }
+
+  errno = 0;
+  for (struct dirent *file; (file = readdir(dir)) != nullptr; errno = 0) {
+    std::filesystem::path absolute_path{path};
+    absolute_path = absolute_path / file->d_name;
+    paths.emplace(absolute_path.string());
+  }
+
+  if (errno != 0) {
+    err_msg << "Expected errno to be 0, instead it is " << errno;
+    throw std::runtime_error(err_msg.str());
+  }
+
+  if (const auto result = closedir(dir); result != 0) {
+    err_msg << "Expected the return value of closedir() to be 0, instead it is "
+            << result;
+    throw std::runtime_error(err_msg.str());
+  }
+  return paths;
+}
+
+TEST_F(DirentCApiTest, TestEmptyFolder) {
+  const auto expected = CreateTempDirAndFiles(0, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestOneFolder) {
+  const auto expected = CreateTempDirAndFiles(1, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestOneFile) {
+  const auto expected = CreateTempDirAndFiles(0, 1);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestMultipleFolders) {
+  const auto expected = CreateTempDirAndFiles(10, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestMultipleFiles) {
+  const auto expected = CreateTempDirAndFiles(0, 10);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestOneFileAndFolder) {
+  const auto expected = CreateTempDirAndFiles(1, 1);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentCApiTest, TestMultipleFilesAndFolders) {
+  const auto expected = CreateTempDirAndFiles(10, 10);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h

@@ -0,0 +1,37 @@
+/**
+ * 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_CROSS_PLATFORM_C_API_DIRENT_TEST
+#define LIBHDFSPP_CROSS_PLATFORM_C_API_DIRENT_TEST
+
+#include "x-platform/dirent.h"
+#include "x-platform/dirent_test.h"
+
+/**
+ * Test fixture for testing {@link XPlatform::Dirent}'s C APIs.
+ */
+class DirentCApiTest : public DirentTest {
+protected:
+  /**
+   * {@inheritdoc}
+   */
+  [[nodiscard]] std::unordered_set<std::string>
+  ListDirAndFiles(const std::string &path) const override;
+};
+
+#endif

+ 151 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc

@@ -0,0 +1,151 @@
+/**
+ * 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 <algorithm>
+#include <filesystem>
+#include <fstream>
+#include <iostream>
+#include <sstream>
+#include <stdexcept>
+#include <string>
+#include <system_error>
+#include <unordered_set>
+#include <variant>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "dirent_test.h"
+#include "x-platform/dirent.h"
+#include "x-platform/syscall.h"
+
+void DirentTest::SetUp() {
+  tmp_root_ = std::filesystem::temp_directory_path() / GetTempName();
+  if (!std::filesystem::create_directories(tmp_root_)) {
+    std::stringstream err_msg;
+    err_msg << "Unable to create temp directory " << tmp_root_.string();
+    throw std::runtime_error(err_msg.str());
+  }
+}
+
+void DirentTest::TearDown() { std::filesystem::remove_all(tmp_root_); }
+
+std::string DirentTest::GetTempName(const std::string &pattern) const {
+  std::vector pattern_raw(pattern.begin(), pattern.end());
+  if (!XPlatform::Syscall::CreateTempDir(pattern_raw)) {
+    std::stringstream err_msg;
+    err_msg << "Creating temp dir failed" << std::endl;
+    throw std::runtime_error(err_msg.str());
+  }
+
+  std::string tmp_dir_path(pattern_raw.data());
+  std::filesystem::remove(tmp_dir_path);
+  return tmp_dir_path;
+}
+
+std::unordered_set<std::string>
+DirentTest::CreateTempDirAndFiles(std::size_t num_dirs,
+                                  std::size_t num_files) const {
+  std::unordered_set<std::string> paths;
+  for (std::size_t i = 0; i < num_dirs; ++i) {
+    const auto tmp_dir_absolute_path = tmp_root_ / std::to_string(i);
+    if (std::error_code err;
+        !std::filesystem::create_directories(tmp_dir_absolute_path, err)) {
+      std::stringstream err_msg;
+      err_msg << "Unable to create the temp dir "
+              << tmp_dir_absolute_path.string() << " reason: " << err.message();
+      throw std::runtime_error(err_msg.str());
+    }
+    paths.emplace(tmp_dir_absolute_path.string());
+  }
+
+  for (std::size_t i = 0; i < num_files; ++i) {
+    std::stringstream tmp_filename;
+    tmp_filename << i << ".txt";
+    const auto tmp_file_absolute_path = tmp_root_ / tmp_filename.str();
+    std::ofstream{tmp_file_absolute_path};
+    paths.emplace(tmp_file_absolute_path.string());
+  }
+  return paths;
+}
+
+std::unordered_set<std::string>
+DirentTest::ListDirAndFiles(const std::string &path) const {
+  std::unordered_set<std::string> paths;
+
+  XPlatform::Dirent dirent(path);
+  for (auto dir_entry = dirent.NextFile();
+       !std::holds_alternative<std::monostate>(dir_entry);
+       dir_entry = dirent.NextFile()) {
+    if (std::holds_alternative<std::error_code>(dir_entry)) {
+      std::stringstream err_msg;
+      const auto err = std::get<std::error_code>(dir_entry);
+      err_msg << "Error in listing directory " << path
+              << " cause: " << err.message();
+      throw std::runtime_error(err_msg.str());
+    }
+
+    if (std::holds_alternative<std::filesystem::directory_entry>(dir_entry)) {
+      const auto entry = std::get<std::filesystem::directory_entry>(dir_entry);
+      paths.emplace(entry.path().string());
+    }
+  }
+  return paths;
+}
+
+TEST_F(DirentTest, TestEmptyFolder) {
+  const auto expected = CreateTempDirAndFiles(0, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestOneFolder) {
+  const auto expected = CreateTempDirAndFiles(1, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestOneFile) {
+  const auto expected = CreateTempDirAndFiles(0, 1);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestMultipleFolders) {
+  const auto expected = CreateTempDirAndFiles(10, 0);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestMultipleFiles) {
+  const auto expected = CreateTempDirAndFiles(0, 10);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestOneFileAndFolder) {
+  const auto expected = CreateTempDirAndFiles(1, 1);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}
+
+TEST_F(DirentTest, TestMultipleFilesAndFolders) {
+  const auto expected = CreateTempDirAndFiles(10, 10);
+  const auto actual = ListDirAndFiles(tmp_root_.string());
+  EXPECT_EQ(expected, actual);
+}

+ 78 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h

@@ -0,0 +1,78 @@
+/**
+ * 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_CROSS_PLATFORM_DIRENT_TEST
+#define LIBHDFSPP_CROSS_PLATFORM_DIRENT_TEST
+
+#include <filesystem>
+#include <string>
+#include <unordered_set>
+
+#include <gtest/gtest.h>
+
+#include "x-platform/dirent.h"
+
+/**
+ * Test fixture for testing {@link XPlatform::Dirent}.
+ */
+class DirentTest : public ::testing::Test {
+protected:
+  void SetUp() override;
+  void TearDown() override;
+
+  /**
+   * Gets a name for creating temporary file or folder. This also ensures that
+   * the temporary file or folder does not exist.
+   *
+   * @param pattern The pattern to use for naming the temporary directory.
+   * @return The temporary file or folder name that can be used for creating the
+   * same.
+   */
+  [[nodiscard]] std::string
+  GetTempName(const std::string &pattern = "test_XXXXXX") const;
+
+  /**
+   * Creates the given number of temporary files and directories under the
+   * {@link DirentTest#tmp_root_}.
+   *
+   * @param num_dirs The number of temporary directories to create.
+   * @param num_files The number of temporary files to create.
+   * @return An {@link std::unordered_set> of the absolute paths of all the
+   * temporary files and folders that were created.
+   */
+  [[nodiscard]] std::unordered_set<std::string>
+  CreateTempDirAndFiles(std::size_t num_dirs, std::size_t num_files) const;
+
+  /**
+   * Lists all the children of the given path.
+   *
+   * @param path The path whose children must be listed.
+   * @return An {@link std::unordered_set} containing the absolute paths of all
+   * the children of the given path.
+   */
+  [[nodiscard]] virtual std::unordered_set<std::string>
+  ListDirAndFiles(const std::string &path) const;
+
+  /**
+   * The root in temp folder under which the rest of all the temporary files and
+   * folders will be created for the purpose of testing.
+   */
+  std::filesystem::path tmp_root_;
+};
+
+#endif