You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "Tom-Newton (via GitHub)" <gi...@apache.org> on 2023/10/29 16:15:59 UTC

[PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Tom-Newton opened a new pull request, #38505:
URL: https://github.com/apache/arrow/pull/38505

   <!--
   Thanks for opening a pull request!
   If this is your first pull request you can find detailed information on how 
   to contribute here:
     * [New Contributor's Guide](https://arrow.apache.org/docs/dev/developers/guide/step_by_step/pr_lifecycle.html#reviews-and-merge-of-the-pull-request)
     * [Contributing Overview](https://arrow.apache.org/docs/dev/developers/overview.html)
   
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/main/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on GitHub? https://github.com/apache/arrow/issues/new/choose
   
   Opening GitHub issues ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename the pull request title in the following format?
   
       GH-${GITHUB_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   In the case of PARQUET issues on JIRA the title also supports:
   
       PARQUET-${JIRA_ISSUE_ID}: [${COMPONENT}] ${SUMMARY}
   
   -->
   
   ### Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   `GetFileInfo` is an important part of an Arrow filesystem implementation. 
   
   ### What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   ### Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   ### Are there any user-facing changes?
   Yes. `GetFileInfo` is now supported on the Azure filesystem. 
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please uncomment the line below and explain which changes are breaking.
   -->
   <!-- **This PR includes breaking changes to public APIs.** -->
   
   <!--
   Please uncomment the line below (and provide explanation) if the changes fix either (a) a security vulnerability, (b) a bug that caused incorrect or invalid data to be produced, or (c) a bug that causes a crash (even when the API contract is upheld). We use this to highlight fixes to issues that may affect users without their knowledge. For this reason, fixing bugs that cause errors don't count, since those are usually obvious.
   -->
   <!-- **This PR contains a "Critical Fix".** -->


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #38505:
URL: https://github.com/apache/arrow/pull/38505#issuecomment-1784156446

   :warning: GitHub issue #38335 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1383022587


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -317,27 +321,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;
 
   explicit Impl(AzureOptions options, io::IOContext io_context)
       : io_context_(io_context), options_(std::move(options)) {}
 
   Status Init() {
-    service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
+    blob_service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
         options_.account_blob_url, options_.storage_credentials_provider);
+    datalake_service_client_ =
+        std::make_shared<Azure::Storage::Files::DataLake::DataLakeServiceClient>(
+            options_.account_dfs_url, options_.storage_credentials_provider);
+    RETURN_NOT_OK(hierarchical_namespace_.Init(datalake_service_client_));
     return Status::OK();
   }
 
   const AzureOptions& options() const { return options_; }
 
+ public:
+  Result<FileInfo> GetFileInfo(const AzurePath& path) {
+    FileInfo info;
+    info.set_path(path.full_path);
+
+    if (path.container.empty()) {
+      DCHECK(path.path_to_file.empty());  // The path is invalid if the container is empty
+                                          // but not path_to_file.
+      // path must refer to the root of the Azure storage account. This is a directory,
+      // and there isn't any extra metadata to fetch.
+      return FileInfo(path.full_path, FileType::Directory);

Review Comment:
   Oops 



##########
cpp/src/arrow/filesystem/path_util.cc:
##########
@@ -191,12 +191,19 @@ std::string_view RemoveLeadingSlash(std::string_view key) {
 }
 
 Status AssertNoTrailingSlash(std::string_view key) {
-  if (key.back() == '/') {
+  if (HasTrailingSlash(key)) {
     return NotAFile(key);
   }
   return Status::OK();
 }
 
+bool HasTrailingSlash(std::string_view key) {
+  if (key.back() != '/') {
+    return false;
+  }
+  return true;

Review Comment:
   :+1: I will make the change to `HasLeadingSlash()` too. (I was thinking the same but chose to copy `HasLeadingSlash()`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on PR #38505:
URL: https://github.com/apache/arrow/pull/38505#issuecomment-1803056798

   The lint failure was fixed by #38639.
   I'll rebase on main before we merge this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on PR #38505:
URL: https://github.com/apache/arrow/pull/38505#issuecomment-1794511091

   Thanks for reviewing kou. I have addressed most of the comments and I should be able to address the remaining ones this evening. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386223441


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -146,9 +148,8 @@ Status ValidateFilePath(const AzurePath& path) {
   return Status::OK();
 }
 
-Status ErrorToStatus(const std::string& prefix,
-                     const Azure::Storage::StorageException& exception) {
-  return Status::IOError(prefix, " Azure Error: ", exception.what());
+bool ContainerOrBlobNotFound(const Azure::Storage::StorageException& exception) {

Review Comment:
   Yeah I'm having second thoughts about this too, I extracted it to a helper function when I thought there was an extra status code I could check to distinguish container and blob not found from other 404s https://github.com/Azure/azure-sdk-for-cpp/blob/e94719131d2ba0e24327f838b74e153af692a46c/sdk/storage/azure-storage-blobs/src/blob_client.cpp#L793-L795. 
   
   Unfortunately it seems blob storage actually doesn't return these error codes consistently on all endpoints so we have to assume 404s are always contaner or blob not found. 
   
   I will revert back to `exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "conbench-apache-arrow[bot] (via GitHub)" <gi...@apache.org>.
conbench-apache-arrow[bot] commented on PR #38505:
URL: https://github.com/apache/arrow/pull/38505#issuecomment-1803298434

   After merging your PR, Conbench analyzed the 5 benchmarking runs that have been run so far on merge-commit 75a04030996261333e843a93f4981156372fa7f8.
   
   There were no benchmark performance regressions. 🎉
   
   The [full Conbench report](https://github.com/apache/arrow/runs/18513693121) has more details. It also includes information about 2 possible false positives for unstable benchmarks that are known to sometimes produce them.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1387402198


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +453,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::unique_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;
 
   explicit Impl(AzureOptions options, io::IOContext io_context)
       : io_context_(io_context), options_(std::move(options)) {}
 
   Status Init() {
-    service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
+    blob_service_client_ = std::make_unique<Azure::Storage::Blobs::BlobServiceClient>(
         options_.account_blob_url, options_.storage_credentials_provider);
+    datalake_service_client_ =
+        std::make_unique<Azure::Storage::Files::DataLake::DataLakeServiceClient>(
+            options_.account_dfs_url, options_.storage_credentials_provider);
+    RETURN_NOT_OK(hierarchical_namespace_.Init(datalake_service_client_.get()));
     return Status::OK();
   }
 
   const AzureOptions& options() const { return options_; }
 
+ public:
+  Result<FileInfo> GetFileInfo(const AzurePath& path) {
+    FileInfo info;
+    info.set_path(path.full_path);
+
+    if (path.container.empty()) {
+      DCHECK(path.path_to_file.empty());  // The path is invalid if the container is empty
+                                          // but not path_to_file.
+      // path must refer to the root of the Azure storage account. This is a directory,
+      // and there isn't any extra metadata to fetch.
+      info.set_type(FileType::Directory);
+      return info;
+    }
+    if (path.path_to_file.empty()) {
+      // path refers to a container. This is a directory if it exists.
+      auto container_client =
+          blob_service_client_->GetBlobContainerClient(path.container);
+      try {
+        auto properties = container_client.GetProperties();
+        info.set_type(FileType::Directory);
+        info.set_mtime(
+            std::chrono::system_clock::time_point(properties.Value.LastModified));
+        return info;
+      } catch (const Azure::Storage::StorageException& exception) {
+        if (exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) {
+          info.set_type(FileType::NotFound);
+          return info;
+        }
+        return internal::ExceptionToStatus(
+            "GetProperties for '" + container_client.GetUrl() +
+                "' failed with an unexpected Azure error. GetFileInfo is unable to "
+                "determine whether the container exists.",
+            exception);
+      }
+    }
+    auto file_client = datalake_service_client_->GetFileSystemClient(path.container)
+                           .GetFileClient(path.path_to_file);
+    try {
+      auto properties = file_client.GetProperties();
+      if (properties.Value.IsDirectory) {
+        info.set_type(FileType::Directory);
+      } else if (internal::HasTrailingSlash(path.path_to_file)) {
+        // For a path with a trailing slash a hierarchical namespace may return a blob
+        // with that trailing slash removed. For consistency with flat namespace and
+        // other filesystems we chose to return NotFound.
+        info.set_type(FileType::NotFound);
+        return info;
+      } else {
+        info.set_type(FileType::File);
+        info.set_size(properties.Value.FileSize);
+      }
+      info.set_mtime(
+          std::chrono::system_clock::time_point(properties.Value.LastModified));
+      return info;
+    } catch (const Azure::Storage::StorageException& exception) {
+      if (exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) {
+        ARROW_ASSIGN_OR_RAISE(bool hierarchical_namespace_enabled,

Review Comment:
   ```suggestion
           ARROW_ASSIGN_OR_RAISE(auto hierarchical_namespace_enabled,
   ```



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   OK. Let's use the approach.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on PR #38505:
URL: https://github.com/apache/arrow/pull/38505#issuecomment-1803123789

   I'll merge this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386291136


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I decided to just make my preferred change. If you think its a bad idea I'm happy to change it again to something else. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386291136


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I decided to just make this change. If you think its a bad idea I'm happy to change it to something else. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386217313


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I made it separate because I wanted to keep the cached value `enabled_` private from the rest of `Impl`. I was a bit concerned that people might try to directly access the cached state without realising that everything should use the `Enabled()` function. 
   
   I think one possibility is to use a non-smart pointer in `HierarchicalNamespaceDetector` because `HierarchicalNamespaceDetector` will always be destructed at the same time as `Impl`. https://stackoverflow.com/questions/7657718/when-to-use-shared-ptr-and-when-to-use-raw-pointers. I think this would be my preferred solution. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1382616550


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -78,18 +81,17 @@ struct AzurePath {
           "Expected an Azure object path of the form 'container/path...', got a URI: '",
           s, "'");
     }
-    const auto src = internal::RemoveTrailingSlash(s);

Review Comment:
   This was preventing `GetFileInfo` working on directories. The other filesystems did not have this.



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +227,223 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real hierachical namespace "
+        "account.");
+  }
+};
+
+TEST_F(AzureFlatNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(true, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespaceFailsWithMissingContainer) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_NOT_OK(hierarchical_namespace.Enabled("non-existent-container"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoAccount) {
+  arrow::fs::AssertFileInfo(fs_.get(), "", FileType::Directory);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoContainer) {
+  arrow::fs::AssertFileInfo(fs_.get(), PreexistingContainerName(), FileType::Directory);
+
+  arrow::fs::AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://" + PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  datalake_service_client_->GetFileSystemClient(PreexistingContainerName())
+      .GetDirectoryClient("test-empty-object-dir")
+      .Create();
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-empty-object-dir",
+                 FileType::Directory);

Review Comment:
   Ideally I would have liked to add an assertion here which confirms that with the hierarchical namespace there are no calls to `ListBlobs`. That would require patching an Azure container client, which I didn't know how to do. If anyone was any suggestions that would be appreciated. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1383043647


##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -308,12 +473,12 @@ TEST_F(TestAzureFileSystem, OpenInputStreamReadMetadata) {
 
   std::shared_ptr<const KeyValueMetadata> actual;
   ASSERT_OK_AND_ASSIGN(actual, stream->ReadMetadata());
-  // TODO(GH-38330): This is asserting that the user defined metadata is returned but this
-  // is probably not the correct behaviour.
+  // TODO(GH-38330): This is asserting that the user defined metadata is returned but
+  // this is probably not the correct behaviour.

Review Comment:
   Will do, though I think there may be other conflicts e.g. renaming `TestAzureFileSystem` -> `AzuriteFileSystemTest`. If you merge #38524 first I can rebase this without too much difficulty. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386217313


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I made it separate because I wanted to keep the cached value `enabled_` private from the rest of `Impl`. I was a bit concerned that people might try to directly access the cached state without realising that everything should use the `Enabled()` function. 
   
   I think one possibility is to use a non-smart pointer in `HierarchicalNamespaceDetector` because `HierarchicalNamespaceDetector` will always be destructed at the same time as `Impl`. https://stackoverflow.com/questions/7657718/when-to-use-shared-ptr-and-when-to-use-raw-pointers. I think that should allow us to use a `unique_ptr` for `datalake_service_client_`. I think this would be my preferred solution. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1384001217


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -317,27 +321,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;

Review Comment:
   👍 If I understand correctly the S3 and GCS filesystems it had to use shared pointers because the top level clients get passed to the input file and output stream classes but for Azure we only pass the blob client to those classes so `blob_service_client_` can beunique. `datalake_service_client_` though is shared with the `HierarchicalNamespaceDetector` so needs to be shared.  



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -317,27 +321,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;

Review Comment:
   👍 If I understand correctly the S3 and GCS filesystems it had to use shared pointers because the top level clients get passed to the input file and output stream classes but for Azure we only pass the blob client to those classes so `blob_service_client_` can beunique. `datalake_service_client_` though is shared with the `HierarchicalNamespaceDetector` so needs to be shared.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386217313


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I made it separate because I wanted to keep the cached value `enabled_` private from the rest of `Impl`. I was a bit concerned that people might try to directly access the cached state rather than using the `Enabled()` function. 
   
   I think one possibility is to use a non-smart pointer in `HierarchicalNamespaceDetector` because `HierarchicalNamespaceDetector` will always be destructed at the same time as `Impl`. https://stackoverflow.com/questions/7657718/when-to-use-shared-ptr-and-when-to-use-raw-pointers. I think this would be my preferred solution. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386217313


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I made it separate because I wanted to keep the cached value `enabled_` private from the rest of `Impl`. I was a bit concerned that people might try to directly access the cached state without realising that everything should use the `Enabled()` function. Additionally making it a separate class made it easier to test. 
   
   I think one possibility is to use a non-smart pointer in `HierarchicalNamespaceDetector` because `HierarchicalNamespaceDetector` will always be destructed at the same time as `Impl`. https://stackoverflow.com/questions/7657718/when-to-use-shared-ptr-and-when-to-use-raw-pointers. I think that should allow us to use a `unique_ptr` for `datalake_service_client_`. I think this would be my preferred solution. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou merged PR #38505:
URL: https://github.com/apache/arrow/pull/38505


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386217313


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   I made is separate because I wanted to keep the cached value `enabled_` private from the rest of `Impl`. I was a bit concerned that people might try to directly access the cached state rather than using the `Enabled()` function. 
   
   I think one possibility is to use a non-smart pointer in `HierarchicalNamespaceDetector` because `HierarchicalNamespaceDetector` will always be destructed at the same time as `Impl`. https://stackoverflow.com/questions/7657718/when-to-use-shared-ptr-and-when-to-use-raw-pointers. I think this would be my preferred solution. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1386223441


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -146,9 +148,8 @@ Status ValidateFilePath(const AzurePath& path) {
   return Status::OK();
 }
 
-Status ErrorToStatus(const std::string& prefix,
-                     const Azure::Storage::StorageException& exception) {
-  return Status::IOError(prefix, " Azure Error: ", exception.what());
+bool ContainerOrBlobNotFound(const Azure::Storage::StorageException& exception) {

Review Comment:
   Yeah I'm having second thoughts about this too, I extracted it to a helper function when I thought there was an extra status code I could check to distinguish container and blob not found from other 404s https://github.com/Azure/azure-sdk-for-cpp/blob/e94719131d2ba0e24327f838b74e153af692a46c/sdk/storage/azure-storage-blobs/src/blob_client.cpp#L793-L795. 
   
   Unfortunately it seems blob storage actually doesn't return these error codes consistently on all endpoints so we have to assume 404s are always contaner or blob not found. 
   
   I will revert back to using `exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound` inline. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1385857556


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;

Review Comment:
   It seems that `HierarchicalNamespaceDetector` is enough simple to move to `Impl`. (`HierarchicalNamespaceDetector::Enabled()` is the only important method in the class.)
   
   How about moving `HierarchicalNamespaceDetector::Enabled()` to `Impl::IsHierarchicalNamespaceEnabled()` and removing `HierarchicalNamespaceDetector` (or something)?
   If we do it, we can make `datalake_service_client_` `std::unique_ptr`.



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -146,9 +148,8 @@ Status ValidateFilePath(const AzurePath& path) {
   return Status::OK();
 }
 
-Status ErrorToStatus(const std::string& prefix,
-                     const Azure::Storage::StorageException& exception) {
-  return Status::IOError(prefix, " Azure Error: ", exception.what());
+bool ContainerOrBlobNotFound(const Azure::Storage::StorageException& exception) {

Review Comment:
   I can understand this naming but I feel that it doesn't add so much information.
   How about just using `IsNotFoundStatus()`, `IsNotFoundException()` or something?
   (Or we don't need to add this helper function because `exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound` is enough easy to understand and straightforward.)



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +227,223 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real hierachical namespace "
+        "account.");
+  }
+};
+
+TEST_F(AzureFlatNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(true, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespaceFailsWithMissingContainer) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_NOT_OK(hierarchical_namespace.Enabled("non-existent-container"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoAccount) {
+  arrow::fs::AssertFileInfo(fs_.get(), "", FileType::Directory);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoContainer) {
+  arrow::fs::AssertFileInfo(fs_.get(), PreexistingContainerName(), FileType::Directory);
+
+  arrow::fs::AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://" + PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  datalake_service_client_->GetFileSystemClient(PreexistingContainerName())
+      .GetDirectoryClient("test-empty-object-dir")
+      .Create();
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-empty-object-dir",
+                 FileType::Directory);

Review Comment:
   We can do it by adding internal `ListBlobs` call counter and exporting it only for testing.
   Or we may be able to provide `AzureFileSystem::GetStatistics()` and the return value provides statistics including the number of `ListBlobs` calles.
   
   (I think that we don't need test it. If we want to test it, we can open a new issue for it and defer it as a separated task to merge this as soon as possible.)



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -453,27 +457,137 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::unique_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;
 
   explicit Impl(AzureOptions options, io::IOContext io_context)
       : io_context_(io_context), options_(std::move(options)) {}
 
   Status Init() {
-    service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
+    blob_service_client_ = std::make_unique<Azure::Storage::Blobs::BlobServiceClient>(
         options_.account_blob_url, options_.storage_credentials_provider);
+    datalake_service_client_ =
+        std::make_shared<Azure::Storage::Files::DataLake::DataLakeServiceClient>(
+            options_.account_dfs_url, options_.storage_credentials_provider);
+    RETURN_NOT_OK(hierarchical_namespace_.Init(datalake_service_client_));
     return Status::OK();
   }
 
   const AzureOptions& options() const { return options_; }
 
+ public:
+  Result<FileInfo> GetFileInfo(const AzurePath& path) {
+    FileInfo info;
+    info.set_path(path.full_path);
+
+    if (path.container.empty()) {
+      DCHECK(path.path_to_file.empty());  // The path is invalid if the container is empty
+                                          // but not path_to_file.
+      // path must refer to the root of the Azure storage account. This is a directory,
+      // and there isn't any extra metadata to fetch.
+      info.set_type(FileType::Directory);
+      return info;
+    }
+    if (path.path_to_file.empty()) {
+      // path refers to a container. This is a directory if it exists.
+      auto container_client =
+          blob_service_client_->GetBlobContainerClient(path.container);
+      try {
+        auto properties = container_client.GetProperties();
+        info.set_type(FileType::Directory);
+        info.set_mtime(
+            std::chrono::system_clock::time_point(properties.Value.LastModified));
+        return info;
+      } catch (const Azure::Storage::StorageException& exception) {
+        if (ContainerOrBlobNotFound(exception)) {
+          info.set_type(FileType::NotFound);
+          return info;
+        }
+        return internal::ExceptionToStatus(
+            "GetProperties for '" + container_client.GetUrl() +
+                "' failed with an unexpected Azure error. GetFileInfo is unable to "
+                "determine whether the container exists.",
+            exception);
+      }
+    }
+    auto file_client = datalake_service_client_->GetFileSystemClient(path.container)
+                           .GetFileClient(path.path_to_file);
+    try {
+      auto properties = file_client.GetProperties();
+      if (properties.Value.IsDirectory) {
+        info.set_type(FileType::Directory);
+      } else if (internal::HasTrailingSlash(path.path_to_file)) {
+        // For a path with a trailing slash a hierarchical namespace may return a blob
+        // with that trailing slash removed. For consistency with flat namespace and
+        // other filesystems we chose to return NotFound.
+        info.set_type(FileType::NotFound);
+        return info;
+      } else {
+        info.set_type(FileType::File);
+        info.set_size(properties.Value.FileSize);
+      }
+      info.set_mtime(
+          std::chrono::system_clock::time_point(properties.Value.LastModified));
+      return info;
+    } catch (const Azure::Storage::StorageException& exception) {
+      if (ContainerOrBlobNotFound(exception)) {
+        ARROW_ASSIGN_OR_RAISE(bool hierarchical_namespace_enabled,
+                              hierarchical_namespace_.Enabled(path.container));
+        if (hierarchical_namespace_enabled) {
+          // If the hierarchical namespace is enabled, then the storage account will have
+          // explicit directories. Neither a file nor a directory was found.
+          info.set_type(FileType::NotFound);
+          return info;
+        }
+        // On flat namespace accounts there are no real directories. Directories are only
+        // implied by using `/` in the blob name.
+        Azure::Storage::Blobs::ListBlobsOptions list_blob_options;
+
+        // If listing the prefix `path.path_to_file` with trailing slash returns at least
+        // one result then `path` refers to an implied directory.
+        auto prefix = internal::EnsureTrailingSlash(path.path_to_file);
+        list_blob_options.Prefix = prefix;
+        // We only need to know if there is at least one result, so minimise page size
+        // for efficiency.
+        list_blob_options.PageSizeHint = 1;
+
+        try {
+          auto paged_list_result =
+              blob_service_client_->GetBlobContainerClient(path.container)
+                  .ListBlobs(list_blob_options);
+          if (paged_list_result.Blobs.size() > 0) {
+            info.set_type(FileType::Directory);
+            return info;
+          } else {
+            info.set_type(FileType::NotFound);
+            return info;
+          }

Review Comment:
   How about simplify this?
   
   ```suggestion
             } else {
               info.set_type(FileType::NotFound);
             }
             return info;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1387288044


##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +227,223 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real hierachical namespace "
+        "account.");
+  }
+};
+
+TEST_F(AzureFlatNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(true, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespaceFailsWithMissingContainer) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_NOT_OK(hierarchical_namespace.Enabled("non-existent-container"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoAccount) {
+  arrow::fs::AssertFileInfo(fs_.get(), "", FileType::Directory);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoContainer) {
+  arrow::fs::AssertFileInfo(fs_.get(), PreexistingContainerName(), FileType::Directory);
+
+  arrow::fs::AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://" + PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  datalake_service_client_->GetFileSystemClient(PreexistingContainerName())
+      .GetDirectoryClient("test-empty-object-dir")
+      .Create();
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-empty-object-dir",
+                 FileType::Directory);

Review Comment:
   I think I'm happy to leave out such an assertion at least initially. If it was python I would have done it seems like mocking in C++ would be more complicated even if I did understand the language :sweat_smile: 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "Tom-Newton (via GitHub)" <gi...@apache.org>.
Tom-Newton commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1384123947


##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));

Review Comment:
   I switched to `RETURN_NOT_OK()` but I don't think it makes a lot of difference because `ConfigureAccountKeyCredentials` actually segfaults somewhere in the Azure SDK if either `account_name` or `account_key` is null. I don't think I can use `ASSERT_` because the function does not return `void`. 
   
   I adjusted the logic slightly so that the tests are skipped if either of `account_name` or `account_key` is null. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] GH-38335: [C++] Implement `GetFileInfo` for a single file in Azure filesystem [arrow]

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #38505:
URL: https://github.com/apache/arrow/pull/38505#discussion_r1382880057


##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -179,12 +180,14 @@ class ObjectInputFile final : public io::RandomAccessFile {
       metadata_ = GetObjectMetadata(properties.Value.Metadata);
       return Status::OK();
     } catch (const Azure::Storage::StorageException& exception) {
-      if (exception.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) {
-        // Could be either container or blob not found.
+      if (ContainerOrBlobNotFound(exception)) {
         return PathNotFound(path_);
       }
-      return ErrorToStatus(
-          "When fetching properties for '" + blob_client_->GetUrl() + "': ", exception);
+      return internal::ErrorToStatus(
+          "GetProperties failed for '" + blob_client_->GetUrl() +
+              "' with an unexpected Azure error. Can not initialise an ObjectInputFile "
+              "without knowing the file size. ",

Review Comment:
   ```suggestion
                 "without knowing the file size.",
   ```



##########
cpp/src/arrow/filesystem/azurefs_internal.h:
##########
@@ -0,0 +1,47 @@
+// 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.
+
+#pragma once
+
+#include <optional>
+
+#include <azure/storage/files/datalake.hpp>
+
+#include "arrow/result.h"
+
+namespace arrow {
+namespace fs {
+namespace internal {

Review Comment:
   ```suggestion
   namespace arrow::fs::internal {
   ```



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -317,27 +321,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
   AzureOptions options_;
+  internal::HierarchicalNamespaceDetector hierarchical_namespace_;
 
   explicit Impl(AzureOptions options, io::IOContext io_context)
       : io_context_(io_context), options_(std::move(options)) {}
 
   Status Init() {
-    service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
+    blob_service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
         options_.account_blob_url, options_.storage_credentials_provider);
+    datalake_service_client_ =
+        std::make_shared<Azure::Storage::Files::DataLake::DataLakeServiceClient>(
+            options_.account_dfs_url, options_.storage_credentials_provider);
+    RETURN_NOT_OK(hierarchical_namespace_.Init(datalake_service_client_));
     return Status::OK();
   }
 
   const AzureOptions& options() const { return options_; }
 
+ public:
+  Result<FileInfo> GetFileInfo(const AzurePath& path) {
+    FileInfo info;
+    info.set_path(path.full_path);
+
+    if (path.container.empty()) {
+      DCHECK(path.path_to_file.empty());  // The path is invalid if the container is empty
+                                          // but not path_to_file.
+      // path must refer to the root of the Azure storage account. This is a directory,
+      // and there isn't any extra metadata to fetch.
+      return FileInfo(path.full_path, FileType::Directory);

Review Comment:
   ```suggestion
         info.set_type(FileType::Directory);
         return info;
   ```



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -137,34 +141,37 @@ TEST(AzureFileSystem, OptionsCompare) {
   EXPECT_TRUE(options.Equals(options));
 }
 
-class TestAzureFileSystem : public ::testing::Test {
+class AzureFileSystemTest : public ::testing::Test {
  public:
   std::shared_ptr<FileSystem> fs_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  AzureOptions options_;
   std::mt19937_64 generator_;
   std::string container_name_;
+  bool suite_skipped_ = false;
 
-  TestAzureFileSystem() : generator_(std::random_device()()) {}
+  AzureFileSystemTest() : generator_(std::random_device()()) {}
 
-  AzureOptions MakeOptions() {
-    const std::string& account_name = GetAzuriteEnv()->account_name();
-    const std::string& account_key = GetAzuriteEnv()->account_key();
-    AzureOptions options;
-    options.backend = AzureBackend::Azurite;
-    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
-    return options;
-  }
+  virtual Result<AzureOptions> MakeOptions() = 0;
 
   void SetUp() override {
-    ASSERT_THAT(GetAzuriteEnv(), NotNull());
-    ASSERT_OK(GetAzuriteEnv()->status());
-
-    container_name_ = RandomChars(32);
     auto options = MakeOptions();
-    service_client_ = std::make_shared<Azure::Storage::Blobs::BlobServiceClient>(
-        options.account_blob_url, options.storage_credentials_provider);
-    ASSERT_OK_AND_ASSIGN(fs_, AzureFileSystem::Make(options));
-    auto container_client = service_client_->GetBlobContainerClient(container_name_);
+    if (options.ok()) {
+      options_ = options.ValueOrDie();

Review Comment:
   ```suggestion
         options_ = *options;
   ```



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {

Review Comment:
   ```suggestion
       if (const auto account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
   ```



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));

Review Comment:
   dittto.



##########
cpp/src/arrow/filesystem/azurefs_internal.h:
##########
@@ -0,0 +1,47 @@
+// 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.
+
+#pragma once
+
+#include <optional>
+
+#include <azure/storage/files/datalake.hpp>
+
+#include "arrow/result.h"
+
+namespace arrow {
+namespace fs {
+namespace internal {
+
+Status ErrorToStatus(const std::string& prefix,
+                     const Azure::Storage::StorageException& exception);
+
+class HierarchicalNamespaceDetector {
+ public:
+  Status Init(std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+                  datalake_service_client);
+  Result<bool> Enabled(const std::string& container_name);
+
+ private:
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::optional<bool> is_hierarchical_namespace_enabled_;

Review Comment:
   How about simplifying this? Because `hierarchical_namespace` is redundant in `HierarchicalNamespaceDetector` class.
   
   ```suggestion
     std::optional<bool> enabled_;
   ```



##########
cpp/src/arrow/filesystem/path_util.cc:
##########
@@ -191,12 +191,19 @@ std::string_view RemoveLeadingSlash(std::string_view key) {
 }
 
 Status AssertNoTrailingSlash(std::string_view key) {
-  if (key.back() == '/') {
+  if (HasTrailingSlash(key)) {
     return NotAFile(key);
   }
   return Status::OK();
 }
 
+bool HasTrailingSlash(std::string_view key) {
+  if (key.back() != '/') {
+    return false;
+  }
+  return true;

Review Comment:
   ```suggestion
     return key.back() == '/';
   ```
   
   (We may want to simplify `HasLeadingSlash()` too.)



##########
cpp/src/arrow/filesystem/azurefs_internal.cc:
##########
@@ -0,0 +1,93 @@
+// 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 "arrow/filesystem/azurefs_internal.h"
+
+#include <azure/storage/files/datalake.hpp>
+
+#include "arrow/result.h"
+
+namespace arrow {
+namespace fs {
+namespace internal {

Review Comment:
   ```suggestion
   namespace arrow::fs::internal {
   ```



##########
cpp/src/arrow/filesystem/azurefs_internal.h:
##########
@@ -0,0 +1,47 @@
+// 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.
+
+#pragma once
+
+#include <optional>
+
+#include <azure/storage/files/datalake.hpp>
+
+#include "arrow/result.h"
+
+namespace arrow {
+namespace fs {
+namespace internal {
+
+Status ErrorToStatus(const std::string& prefix,

Review Comment:
   How about renaming this to `ExceptionToStatus()` because Azure SDK uses `Exception` such as `StorageException`?



##########
cpp/src/arrow/filesystem/azurefs.cc:
##########
@@ -317,27 +321,136 @@ class ObjectInputFile final : public io::RandomAccessFile {
 class AzureFileSystem::Impl {
  public:
   io::IOContext io_context_;
-  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> service_client_;
+  std::shared_ptr<Azure::Storage::Files::DataLake::DataLakeServiceClient>
+      datalake_service_client_;
+  std::shared_ptr<Azure::Storage::Blobs::BlobServiceClient> blob_service_client_;

Review Comment:
   It seems that this isn't shared.
   How about using `std::unique_ptr` instead?



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -308,12 +473,12 @@ TEST_F(TestAzureFileSystem, OpenInputStreamReadMetadata) {
 
   std::shared_ptr<const KeyValueMetadata> actual;
   ASSERT_OK_AND_ASSIGN(actual, stream->ReadMetadata());
-  // TODO(GH-38330): This is asserting that the user defined metadata is returned but this
-  // is probably not the correct behaviour.
+  // TODO(GH-38330): This is asserting that the user defined metadata is returned but
+  // this is probably not the correct behaviour.

Review Comment:
   Could you revert this change?
   This will be conflicted with #38524.



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");

Review Comment:
   ```suggestion
         const auto account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
   ```



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real hierachical namespace "
+        "account.");
+  }
+};
+
+TEST_F(AzureFlatNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(true, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespaceFailsWithMissingContainer) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_NOT_OK(hierarchical_namespace.Enabled("non-existent-container"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoAccount) {
+  arrow::fs::AssertFileInfo(fs_.get(), "", FileType::Directory);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoContainer) {
+  arrow::fs::AssertFileInfo(fs_.get(), PreexistingContainerName(), FileType::Directory);
+
+  arrow::fs::AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);

Review Comment:
   ```suggestion
     AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);
   ```



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));

Review Comment:
   How about using `RETURN_NOT_OK()` or `ASSERT_OK()` to skip or stop the test on failure?



##########
cpp/src/arrow/filesystem/azurefs_test.cc:
##########
@@ -216,23 +226,184 @@ class TestAzureFileSystem : public ::testing::Test {
   void UploadLines(const std::vector<std::string>& lines, const char* path_to_file,
                    int total_size) {
     // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
-    auto blob_client = service_client_->GetBlobContainerClient(PreexistingContainerName())
-                           .GetBlockBlobClient(path_to_file);
+    auto blob_client =
+        blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+            .GetBlockBlobClient(path_to_file);
     std::string all_lines = std::accumulate(lines.begin(), lines.end(), std::string(""));
     blob_client.UploadFrom(reinterpret_cast<const uint8_t*>(all_lines.data()),
                            total_size);
   }
+
+  void RunGetFileInfoObjectWithNestedStructureTest();
+  void RunGetFileInfoObjectTest();
 };
 
-TEST_F(TestAzureFileSystem, OpenInputStreamString) {
+class AzuriteFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() {
+    EXPECT_THAT(GetAzuriteEnv(), NotNull());
+    ARROW_EXPECT_OK(GetAzuriteEnv()->status());
+    AzureOptions options;
+    options.backend = AzureBackend::Azurite;
+    ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(
+        GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()));
+    return options;
+  }
+};
+
+class AzureFlatNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_FLAT_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real flat namespace "
+        "account.");
+  }
+};
+
+class AzureHierarchicalNamespaceFileSystemTest : public AzureFileSystemTest {
+  Result<AzureOptions> MakeOptions() override {
+    AzureOptions options;
+    if (char* account_name = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_NAME")) {
+      char* account_key = std::getenv("AZURE_HIERARCHICAL_NAMESPACE_ACCOUNT_KEY");
+      EXPECT_THAT(account_key, NotNull());
+      ARROW_EXPECT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key));
+      return options;
+    }
+    return Status::Cancelled(
+        "Connection details not provided for a real hierachical namespace "
+        "account.");
+  }
+};
+
+TEST_F(AzureFlatNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(true, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespace) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_OK_AND_EQ(false, hierarchical_namespace.Enabled(PreexistingContainerName()));
+}
+
+TEST_F(AzuriteFileSystemTest, DetectHierarchicalNamespaceFailsWithMissingContainer) {
+  auto hierarchical_namespace = internal::HierarchicalNamespaceDetector();
+  ASSERT_OK(hierarchical_namespace.Init(datalake_service_client_));
+  ASSERT_NOT_OK(hierarchical_namespace.Enabled("non-existent-container"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoAccount) {
+  arrow::fs::AssertFileInfo(fs_.get(), "", FileType::Directory);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://"));
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoContainer) {
+  arrow::fs::AssertFileInfo(fs_.get(), PreexistingContainerName(), FileType::Directory);
+
+  arrow::fs::AssertFileInfo(fs_.get(), "non-existent-container", FileType::NotFound);
+
+  // URI
+  ASSERT_RAISES(Invalid, fs_->GetFileInfo("abfs://" + PreexistingContainerName()));
+}
+
+void AzureFileSystemTest::RunGetFileInfoObjectWithNestedStructureTest() {
+  // Adds detailed tests to handle cases of different edge cases
+  // with directory naming conventions (e.g. with and without slashes).
+  constexpr auto kObjectName = "test-object-dir/some_other_dir/another_dir/foo";
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(kObjectName)
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  // 0 is immediately after "/" lexicographically, ensure that this doesn't
+  // cause unexpected issues.
+  // TODO(GH-38333): Switch to using Azure filesystem to write once its implemented.
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient("test-object-dir/some_other_dir0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+      .GetBlockBlobClient(std::string(kObjectName) + "0")
+      .UploadFrom(reinterpret_cast<const uint8_t*>(kLoremIpsum), strlen(kLoremIpsum));
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName, FileType::File);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + kObjectName + "/",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_dir",
+                 FileType::Directory);
+  AssertFileInfo(fs_.get(),
+                 PreexistingContainerPath() + "test-object-dir/some_other_dir/",
+                 FileType::Directory);
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-di",
+                 FileType::NotFound);
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-object-dir/some_other_di",
+                 FileType::NotFound);
+}
+
+TEST_F(AzuriteFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  RunGetFileInfoObjectWithNestedStructureTest();
+}
+
+TEST_F(AzureHierarchicalNamespaceFileSystemTest, GetFileInfoObjectWithNestedStructure) {
+  RunGetFileInfoObjectWithNestedStructureTest();
+  datalake_service_client_->GetFileSystemClient(PreexistingContainerName())
+      .GetDirectoryClient("test-empty-object-dir")
+      .Create();
+
+  AssertFileInfo(fs_.get(), PreexistingContainerPath() + "test-empty-object-dir",
+                 FileType::Directory);
+}
+
+void AzureFileSystemTest::RunGetFileInfoObjectTest() {
+  auto object_properties =
+      blob_service_client_->GetBlobContainerClient(PreexistingContainerName())
+          .GetBlobClient(PreexistingObjectName())
+          .GetProperties()
+          .Value;
+
+  arrow::fs::AssertFileInfo(

Review Comment:
   ```suggestion
     AssertFileInfo(
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org