From b60c7f437586e65f9867b72988aa8064d3b954e2 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 18 Apr 2022 15:02:51 +0530 Subject: [PATCH 01/34] ARROW-2034: [C++] Filesystem implementation for AzureBlobFileSystem --- cpp/CMakeLists.txt | 8 + cpp/cmake_modules/BuildUtils.cmake | 254 +++ cpp/cmake_modules/DefineOptions.cmake | 2 + cpp/cmake_modules/ThirdpartyToolchain.cmake | 47 + cpp/src/arrow/CMakeLists.txt | 45 + cpp/src/arrow/filesystem/azure/CMakeLists.txt | 36 + cpp/src/arrow/filesystem/azure/azurefs.cc | 1546 +++++++++++++++++ cpp/src/arrow/filesystem/azure/azurefs.h | 157 ++ .../arrow/filesystem/azure/azurefs_test.cc | 1096 ++++++++++++ cpp/vcpkg.json | 5 + 10 files changed, 3196 insertions(+) create mode 100644 cpp/src/arrow/filesystem/azure/CMakeLists.txt create mode 100644 cpp/src/arrow/filesystem/azure/azurefs.cc create mode 100644 cpp/src/arrow/filesystem/azure/azurefs.h create mode 100644 cpp/src/arrow/filesystem/azure/azurefs_test.cc diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index ffa5cc566005a..7b2d941336dda 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -693,7 +693,9 @@ endif() # Libraries to link statically with libarrow.so set(ARROW_LINK_LIBS) +set(ARROW_AZURE_LINK_LIBS) set(ARROW_STATIC_LINK_LIBS) +set(ARROW_AZURE_STATIC_LINK_LIBS) set(ARROW_STATIC_INSTALL_INTERFACE_LIBS) if(ARROW_USE_OPENSSL) @@ -792,6 +794,11 @@ if(ARROW_WITH_OPENTELEMETRY) opentelemetry-cpp::otlp_http_exporter) endif() +if (ARROW_AZURE) + list(APPEND ARROW_AZURE_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_AZURE_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) +endif() + if(ARROW_WITH_UTF8PROC) list(APPEND ARROW_LINK_LIBS utf8proc::utf8proc) list(APPEND ARROW_STATIC_LINK_LIBS utf8proc::utf8proc) @@ -829,6 +836,7 @@ if(ARROW_STATIC_LINK_LIBS) endif() set(ARROW_SHARED_PRIVATE_LINK_LIBS ${ARROW_STATIC_LINK_LIBS}) +set(ARROW_AZURE_SHARED_PRIVATE_LINK_LIBS ${ARROW_AZURE_STATIC_LINK_LIBS}) # boost::filesystem is needed for S3 and Flight tests as a boost::process dependency. if(((ARROW_FLIGHT diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake index 174b1c515a78b..42e45b8b02f4a 100644 --- a/cpp/cmake_modules/BuildUtils.cmake +++ b/cpp/cmake_modules/BuildUtils.cmake @@ -46,6 +46,260 @@ if(WIN32 AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") list(APPEND ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS "BOOST_USE_WINDOWS_H=1") endif() +function(ADD_ARROW_LIB_AZURE LIB_NAME) + set(options) + set(one_value_args + BUILD_SHARED + BUILD_STATIC) + set(multi_value_args + SOURCES + STATIC_LINK_LIBS + SHARED_LINK_LIBS + DEPENDENCIES + SHARED_PRIVATE_LINK_LIBS + OUTPUT_PATH) + cmake_parse_arguments(ARG + "${options}" + "${one_value_args}" + "${multi_value_args}" + ${ARGN}) + if(ARG_UNPARSED_ARGUMENTS) + message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}") + endif() + + if(ARG_SOURCES) + set(SOURCES ${ARG_SOURCES}) + else() + set(SOURCES "${LIB_NAME}.cc") + endif() + + # Allow overriding ARROW_BUILD_SHARED and ARROW_BUILD_STATIC + if(DEFINED ARG_BUILD_SHARED) + set(BUILD_SHARED ${ARG_BUILD_SHARED}) + else() + set(BUILD_SHARED ${ARROW_BUILD_SHARED}) + endif() + if(DEFINED ARG_BUILD_STATIC) + set(BUILD_STATIC ${ARG_BUILD_STATIC}) + else() + set(BUILD_STATIC ${ARROW_BUILD_STATIC}) + endif() + if(ARG_OUTPUT_PATH) + set(OUTPUT_PATH ${ARG_OUTPUT_PATH}) + else() + set(OUTPUT_PATH ${BUILD_OUTPUT_ROOT_DIRECTORY}) + endif() + + if(WIN32 OR (CMAKE_GENERATOR STREQUAL Xcode)) + # We need to compile C++ separately for each library kind (shared and static) + # because of dllexport declarations on Windows. + # The Xcode generator doesn't reliably work with Xcode as target names are not + # guessed correctly. + set(USE_OBJLIB OFF) + else() + set(USE_OBJLIB ON) + endif() + + if(USE_OBJLIB) + # Generate a single "objlib" from all C++ modules and link + # that "objlib" into each library kind, to avoid compiling twice + add_library(${LIB_NAME}_objlib OBJECT ${SOURCES}) + # Necessary to make static linking into other shared libraries work properly + set_property(TARGET ${LIB_NAME}_objlib PROPERTY POSITION_INDEPENDENT_CODE 1) + set(LIB_DEPS $) + else() + set(LIB_DEPS ${ARG_SOURCES}) + endif() + + set(RUNTIME_INSTALL_DIR bin) + + if(BUILD_SHARED) + add_library(${LIB_NAME}_shared SHARED ${LIB_DEPS}) + + set_target_properties(${LIB_NAME}_shared + PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${OUTPUT_PATH}" + RUNTIME_OUTPUT_DIRECTORY "${OUTPUT_PATH}" + PDB_OUTPUT_DIRECTORY "${OUTPUT_PATH}" + OUTPUT_NAME ${LIB_NAME} + VERSION "${ARROW_FULL_SO_VERSION}" + SOVERSION "${ARROW_SO_VERSION}") + + target_link_libraries(${LIB_NAME}_shared LINK_PRIVATE ${ARG_SHARED_PRIVATE_LINK_LIBS}) + + install(TARGETS ${LIB_NAME}_shared + EXPORT ${LIB_NAME}_targets + RUNTIME DESTINATION ${RUNTIME_INSTALL_DIR} + LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} + ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} + INCLUDES + DESTINATION ${CMAKE_INSTALL_INCLUDEDIR}) + endif() + + if(BUILD_STATIC) + add_library(${LIB_NAME}_static SHARED ${LIB_DEPS}) + + if(MSVC_TOOLCHAIN) + set(LIB_NAME_STATIC ${LIB_NAME}_static) + else() + set(LIB_NAME_STATIC ${LIB_NAME}) + endif() + + if(ARROW_BUILD_STATIC AND WIN32) + target_compile_definitions(${LIB_NAME}_static PUBLIC ARROW_STATIC) + endif() + + set_target_properties(${LIB_NAME}_static + PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${OUTPUT_PATH}" + OUTPUT_NAME ${LIB_NAME_STATIC}) + + install(TARGETS ${LIB_NAME}_static + EXPORT ${LIB_NAME}_targets + RUNTIME DESTINATION ${RUNTIME_INSTALL_DIR} + LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} + ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} + INCLUDES + DESTINATION ${CMAKE_INSTALL_INCLUDEDIR}) + endif() +endfunction() + +function(ADD_TEST_CASE_AZURE REL_TEST_NAME) + set(options NO_VALGRIND ENABLED) + set(one_value_args PRECOMPILED_HEADER_LIB) + set(multi_value_args + SOURCES + PRECOMPILED_HEADERS + STATIC_LINK_LIBS + EXTRA_LINK_LIBS + EXTRA_INCLUDES + EXTRA_DEPENDENCIES + LABELS + EXTRA_LABELS + TEST_ARGUMENTS + PREFIX) + cmake_parse_arguments(ARG + "${options}" + "${one_value_args}" + "${multi_value_args}" + ${ARGN}) + if(ARG_UNPARSED_ARGUMENTS) + message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}") + endif() + + if(NO_TESTS AND NOT ARG_ENABLED) + return() + endif() + get_filename_component(TEST_NAME ${REL_TEST_NAME} NAME_WE) + + if(ARG_PREFIX) + set(TEST_NAME "${ARG_PREFIX}-${TEST_NAME}") + endif() + + if(ARG_SOURCES) + set(SOURCES ${ARG_SOURCES}) + else() + set(SOURCES "${REL_TEST_NAME}.cc") + endif() + + # Make sure the executable name contains only hyphens, not underscores + string(REPLACE "_" "-" TEST_NAME ${TEST_NAME}) + + set(TEST_PATH "${EXECUTABLE_OUTPUT_PATH}/${TEST_NAME}") + add_executable(${TEST_NAME} ${SOURCES}) + + # target_link_libraries(${TEST_NAME} PRIVATE azurefs_shared) + # With OSX and conda, we need to set the correct RPATH so that dependencies + # are found. The installed libraries with conda have an RPATH that matches + # for executables and libraries lying in $ENV{CONDA_PREFIX}/bin or + # $ENV{CONDA_PREFIX}/lib but our test libraries and executables are not + # installed there. + if(NOT "$ENV{CONDA_PREFIX}" STREQUAL "" AND APPLE) + set_target_properties(${TEST_NAME} + PROPERTIES BUILD_WITH_INSTALL_RPATH TRUE + INSTALL_RPATH_USE_LINK_PATH TRUE + INSTALL_RPATH + "${EXECUTABLE_OUTPUT_PATH};$ENV{CONDA_PREFIX}/lib") + endif() + + if(ARG_STATIC_LINK_LIBS) + # Customize link libraries + target_link_libraries(${TEST_NAME} PRIVATE ${ARG_STATIC_LINK_LIBS}) + else() + target_link_libraries(${TEST_NAME} PRIVATE ${ARROW_TEST_LINK_LIBS}) + endif() + + if(ARG_PRECOMPILED_HEADER_LIB) + reuse_precompiled_header_lib(${TEST_NAME} ${ARG_PRECOMPILED_HEADER_LIB}) + endif() + + if(ARG_PRECOMPILED_HEADERS AND ARROW_USE_PRECOMPILED_HEADERS) + target_precompile_headers(${TEST_NAME} PRIVATE ${ARG_PRECOMPILED_HEADERS}) + endif() + + if(ARG_EXTRA_LINK_LIBS) + target_link_libraries(${TEST_NAME} PRIVATE ${ARG_EXTRA_LINK_LIBS}) + endif() + + if(ARG_EXTRA_INCLUDES) + target_include_directories(${TEST_NAME} SYSTEM PUBLIC ${ARG_EXTRA_INCLUDES}) + endif() + + if(ARG_EXTRA_DEPENDENCIES) + add_dependencies(${TEST_NAME} ${ARG_EXTRA_DEPENDENCIES}) + endif() + + if(ARROW_TEST_MEMCHECK AND NOT ARG_NO_VALGRIND) + add_test(${TEST_NAME} + bash + -c + "cd '${CMAKE_SOURCE_DIR}'; \ + valgrind --suppressions=valgrind.supp --tool=memcheck --gen-suppressions=all \ + --num-callers=500 --leak-check=full --leak-check-heuristics=stdstring \ + --error-exitcode=1 ${TEST_PATH} ${ARG_TEST_ARGUMENTS}") + elseif(WIN32) + add_test(${TEST_NAME} ${TEST_PATH} ${ARG_TEST_ARGUMENTS}) + else() + add_test(${TEST_NAME} + ${BUILD_SUPPORT_DIR}/run-test.sh + ${CMAKE_BINARY_DIR} + test + ${TEST_PATH} + ${ARG_TEST_ARGUMENTS}) + endif() + + # Add test as dependency of relevant targets + add_dependencies(all-tests ${TEST_NAME}) + foreach(TARGET ${ARG_LABELS}) + add_dependencies(${TARGET} ${TEST_NAME}) + endforeach() + + set(LABELS) + list(APPEND LABELS "unittest") + if(ARG_LABELS) + list(APPEND LABELS ${ARG_LABELS}) + endif() + # EXTRA_LABELS don't create their own dependencies, they are only used + # to ease running certain test categories. + if(ARG_EXTRA_LABELS) + list(APPEND LABELS ${ARG_EXTRA_LABELS}) + endif() + + foreach(LABEL ${ARG_LABELS}) + # ensure there is a cmake target which exercises tests with this LABEL + set(LABEL_TEST_NAME "test-${LABEL}") + if(NOT TARGET ${LABEL_TEST_NAME}) + add_custom_target(${LABEL_TEST_NAME} + ctest -L "${LABEL}" --output-on-failure + USES_TERMINAL) + endif() + # ensure the test is (re)built before the LABEL test runs + add_dependencies(${LABEL_TEST_NAME} ${TEST_NAME}) + endforeach() + + set_property(TEST ${TEST_NAME} + APPEND + PROPERTY LABELS ${LABELS}) +endfunction() + function(ADD_THIRDPARTY_LIB LIB_NAME) set(options) set(one_value_args SHARED_LIB STATIC_LIB) diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index ec1e0b6352a36..96701c1cab831 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -215,6 +215,8 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") #---------------------------------------------------------------------- set_option_category("Project component") + define_option(ARROW_AZURE "Build Arrow with Azure support (requires the Azure SDK for C++)" OFF) + define_option(ARROW_BUILD_UTILITIES "Build Arrow commandline utilities" OFF) define_option(ARROW_COMPUTE "Build the Arrow Compute Modules" OFF) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index 9d9376bb8f253..667c200b6e878 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4513,6 +4513,53 @@ if(ARROW_S3) endif() endif() +macro(build_azuresdk) + message(STATUS "Building Azure C++ SDK from source") + + set(AZURESDK_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/azuresdk_ep-install") + set(AZURESDK_INCLUDE_DIR "${AZURESDK_PREFIX}/include") + + set(AZURESDK_CMAKE_ARGS + ${EP_COMMON_CMAKE_ARGS} + -DBUILD_TESTING=OFF + -DCMAKE_INSTALL_LIBDIR=lib + "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" + -DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}) + + file(MAKE_DIRECTORY ${AZURESDK_INCLUDE_DIR}) + + # Azure C++ SDK related libraries to link statically + set(_AZURESDK_LIBS + azure-core + azure-identity + azure-storage-blobs + azure-storage-common + azure-storage-files-datalake) + set(AZURESDK_LIBRARIES) + set(AZURESDK_LIBRARIES_CPP) + foreach(_AZURESDK_LIB ${_AZURESDK_LIBS}) + string(TOUPPER ${_AZURESDK_LIB} _AZURESDK_LIB_UPPER) + string(REPLACE "-" "_" _AZURESDK_LIB_NAME_PREFIX ${_AZURESDK_LIB_UPPER}) + list(APPEND AZURESDK_LIBRARIES_CPP "${_AZURESDK_LIB}-cpp") + set(_AZURESDK_TARGET_NAME Azure::${_AZURESDK_LIB}) + list(APPEND AZURESDK_LIBRARIES ${_AZURESDK_TARGET_NAME}) + endforeach() + + set(AZURESDK_LINK_LIBRARIES ${AZURESDK_LIBRARIES}) +endmacro() + +if(ARROW_AZURE) + build_azuresdk() + + foreach(AZURESDK_LIBRARY_CPP ${AZURESDK_LIBRARIES_CPP}) + find_package(${AZURESDK_LIBRARY_CPP} CONFIG REQUIRED) + endforeach() + + include_directories(SYSTEM ${AZURESDK_INCLUDE_DIR}) + message(STATUS "Found AZURE SDK headers: ${AZURESDK_INCLUDE_DIR}") + message(STATUS "Found AZURE SDK libraries: ${AZURESDK_LINK_LIBRARIES}") +endif() + message(STATUS "All bundled static libraries: ${ARROW_BUNDLED_STATIC_LIBS}") # Write out the package configurations. diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 2933457287407..9f26b7b4747b6 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -68,6 +68,47 @@ function(ADD_ARROW_TEST REL_TEST_NAME) ${ARG_UNPARSED_ARGUMENTS}) endfunction() +function(ADD_ARROW_TEST_AZURE REL_TEST_NAME) + set(options) + set(one_value_args PREFIX) + set(multi_value_args LABELS PRECOMPILED_HEADERS) + cmake_parse_arguments(ARG + "${options}" + "${one_value_args}" + "${multi_value_args}" + ${ARGN}) + + if(ARG_PREFIX) + set(PREFIX ${ARG_PREFIX}) + else() + set(PREFIX "arrow") + endif() + + if(ARG_LABELS) + set(LABELS ${ARG_LABELS}) + else() + set(LABELS "arrow-tests") + endif() + + # Because of https://gitlab.kitware.com/cmake/cmake/issues/20289, + # we must generate the precompiled header on an executable target. + # Do that on the first unit test target (here "arrow-array-test") + # and reuse the PCH for the other tests. + if(ARG_PRECOMPILED_HEADERS) + set(PCH_ARGS PRECOMPILED_HEADERS ${ARG_PRECOMPILED_HEADERS}) + else() + set(PCH_ARGS PRECOMPILED_HEADER_LIB "arrow-array-test") + endif() + + add_test_case_azure(${REL_TEST_NAME} + PREFIX + ${PREFIX} + LABELS + ${LABELS} + ${PCH_ARGS} + ${ARG_UNPARSED_ARGUMENTS}) +endfunction() + function(ADD_ARROW_FUZZ_TARGET REL_FUZZING_NAME) set(options) set(one_value_args PREFIX) @@ -485,6 +526,10 @@ if(ARROW_FILESYSTEM) SKIP_UNITY_BUILD_INCLUSION ON) endif() + if(ARROW_AZURE) + add_subdirectory(filesystem/azure) + endif() + list(APPEND ARROW_TESTING_SRCS filesystem/test_util.cc) endif() diff --git a/cpp/src/arrow/filesystem/azure/CMakeLists.txt b/cpp/src/arrow/filesystem/azure/CMakeLists.txt new file mode 100644 index 0000000000000..a3c0a43e3cd65 --- /dev/null +++ b/cpp/src/arrow/filesystem/azure/CMakeLists.txt @@ -0,0 +1,36 @@ +# 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. + +set(CMAKE_CXX_STANDARD 14) +set(CMAKE_CXX_STANDARD_REQUIRED ON) + +add_arrow_lib_azure(azurefs + SHARED_LINK_LIBS + ${ARROW_AZURE_LINK_LIBS} + SHARED_PRIVATE_LINK_LIBS + ${ARROW_AZURE_SHARED_PRIVATE_LINK_LIBS} + STATIC_LINK_LIBS + ${ARROW_AZURE_STATIC_LINK_LIBS}) + +arrow_install_all_headers("arrow/filesystem/azure") + +set(TEST_LIBS_AZURE ${ARROW_TEST_LINK_LIBS}) +list(APPEND TEST_LIBS_AZURE azurefs_shared) +add_arrow_test_azure(azurefs_test EXTRA_LABELS filesystem + STATIC_LINK_LIBS + ${TEST_LIBS_AZURE} + ) \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/azure/azurefs.cc b/cpp/src/arrow/filesystem/azure/azurefs.cc new file mode 100644 index 0000000000000..583f70ee3f64c --- /dev/null +++ b/cpp/src/arrow/filesystem/azure/azurefs.cc @@ -0,0 +1,1546 @@ +// 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/azure/azurefs.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#ifdef _WIN32 +// Undefine preprocessor macros that interfere with AWS function / method names +#ifdef GetMessage +#undef GetMessage +#endif +#ifdef GetObject +#undef GetObject +#endif +#endif + +#include "arrow/util/windows_fixup.h" + +#include "arrow/buffer.h" +#include "arrow/filesystem/filesystem.h" +#include "arrow/filesystem/path_util.h" +#include "arrow/filesystem/util_internal.h" +#include "arrow/io/interfaces.h" +#include "arrow/io/memory.h" +#include "arrow/io/util_internal.h" +#include "arrow/result.h" +#include "arrow/status.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/atomic_shared_ptr.h" +#include "arrow/util/checked_cast.h" +#include "arrow/util/future.h" +#include "arrow/util/key_value_metadata.h" +#include "arrow/util/logging.h" +#include "arrow/util/optional.h" +#include "arrow/util/task_group.h" +#include "arrow/util/thread_pool.h" + + +namespace arrow { + +using internal::Uri; + +namespace fs { + +static const char kSep = '/'; + +// ----------------------------------------------------------------------- +// AzureOptions implementation + +AzureOptions::AzureOptions() {} + +void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name) { + account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; + account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + credentials_kind = AzureCredentialsKind::Anonymous; +} + +void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, + const std::string& account_key) { + account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; + account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + storage_credentials_provider = std::make_shared(account_name, account_key); + credentials_kind = AzureCredentialsKind::StorageCredentials; +} + +void AzureOptions::ConfigureConnectionStringCredentials(const std::string& connection_string_uri) { + auto account_name = Azure::Storage::_internal::ParseConnectionString(connection_string_uri).AccountName; + account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; + account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + connection_string = connection_string_uri; + credentials_kind = AzureCredentialsKind::ConnectionString; +} + +void AzureOptions::ConfigureServicePrincipleCredentials(const std::string& account_name, + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret) { + account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; + account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + service_principle_credentials_provider = std::make_shared(tenant_id, client_id, client_secret); + credentials_kind = AzureCredentialsKind::ServicePrincipleCredentials; +} + +void AzureOptions::ConfigureSasCredentials(const std::string& uri) { + auto src = internal::RemoveTrailingSlash(uri); + auto first_sep = src.find_first_of("?"); + sas_token = std::string(src.substr(first_sep)); + account_blob_url = std::string(src.substr(0, first_sep)); + src = internal::RemoveTrailingSlash(account_blob_url); + first_sep = src.find("blob.core.windows.net"); + account_dfs_url = std::string(src.substr(0, first_sep)) + "dfs" + std::string(src.substr(first_sep + 4)) +"/"; + credentials_kind = AzureCredentialsKind::Sas; +} + +bool AzureOptions::Equals(const AzureOptions& other) const { + return (scheme == other.scheme && + account_dfs_url == other.account_dfs_url && + account_blob_url == other.account_blob_url && + credentials_kind == other.credentials_kind); +} + +AzureOptions AzureOptions::FromAnonymous(const std::string account_name) { + AzureOptions options; + options.ConfigureAnonymousCredentials(account_name); + return options; +} + +AzureOptions AzureOptions::FromAccountKey(const std::string& account_name, + const std::string& account_key) { + AzureOptions options; + options.ConfigureAccountKeyCredentials(account_name, account_key); + return options; +} + +AzureOptions AzureOptions::FromConnectionString(const std::string& connection_string) { + AzureOptions options; + options.ConfigureConnectionStringCredentials(connection_string); + return options; +} + +AzureOptions AzureOptions::FromServicePrincipleCredential(const std::string& account_name, + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret) { + AzureOptions options; + options.ConfigureServicePrincipleCredentials(account_name, tenant_id, client_id, client_secret); + return options; +} + +AzureOptions AzureOptions::FromSas(const std::string& uri) { + AzureOptions options; + options.ConfigureSasCredentials(uri); + return options; +} + +Result AzureOptions::FromUri(const std::string& uri_string, + std::string* out_path) { + Uri uri; + RETURN_NOT_OK(uri.Parse(uri_string)); + return FromUri(uri, out_path); +} + +Result AzureOptions::FromUri(const Uri& uri, + std::string* out_path) { + AzureOptions options; + AZURE_ASSERT(uri.has_host()); + const auto container = uri.host(); + auto path = uri.path(); + if (container.empty()) { + if (!path.empty()) { + return Status::Invalid("Missing container in URI"); + } + } else { + if (path.empty()) { + path = container.substr(0, container.find('.')); + } else { + if (path[0] != '/') { + return Status::Invalid("URI should absolute, not relative"); + } + path = container + path; + } + } + if (out_path != nullptr) { + *out_path = std::string(internal::RemoveTrailingSlash(path)); + } + + options.scheme = uri.scheme(); + AZURE_ASSERT(container.find('.') != std::string::npos); + std::string accountName = container.substr(0, container.find('.')); + const auto query_string = uri.query_string(); + if (!query_string.empty()) { + options.ConfigureSasCredentials(uri.scheme() + "://" + path + "?" + query_string); + } else { + options.ConfigureAnonymousCredentials(accountName); + } + return options; +} + +namespace { + +struct AzurePath { + std::string full_path; + std::string container; + std::string path_to_file; + std::vector path_to_file_parts; + + static Result FromString(const std::string& s) { + // https://synapsemladlsgen2.dfs.core.windows.net/synapsemlfs/testdir/testfile.txt + // container = synapsemlfs + // account_name = synapsemladlsgen2 + // path_to_file = testdir/testfile.txt + // path_to_file_parts = [testdir, testfile.txt] + + // Expected input here => s = /synapsemlfs/testdir/testfile.txt + auto src = internal::RemoveTrailingSlash(s); + if (src.starts_with("https:") || src.starts_with("http::")) { + RemoveSchemeFromUri(src); + } + auto first_sep = src.find_first_of(kSep); + if (first_sep == 0) { + return Status::Invalid("Path cannot start with a separator ('", s, "')"); + } + if (first_sep == std::string::npos) { + return AzurePath{std::string(src), std::string(src), "", {}}; + } + AzurePath path; + path.full_path = std::string(src); + path.container = std::string(src.substr(0, first_sep)); + path.path_to_file = std::string(src.substr(first_sep + 1)); + path.path_to_file_parts = internal::SplitAbstractPath(path.path_to_file); + RETURN_NOT_OK(Validate(&path)); + return path; + } + + static void RemoveSchemeFromUri(nonstd::sv_lite::string_view& s) { + auto first = s.find(".core.windows.net"); + s = s.substr(first + 18, s.length()); + } + + static Status Validate(const AzurePath* path) { + auto result = internal::ValidateAbstractPathParts(path->path_to_file_parts); + if (!result.ok()) { + return Status::Invalid(result.message(), " in path ", path->full_path); + } else { + return result; + } + } + + AzurePath parent() const { + DCHECK(!path_to_file_parts.empty()); + auto parent = AzurePath{"", container, "", path_to_file_parts}; + parent.path_to_file_parts.pop_back(); + parent.path_to_file = internal::JoinAbstractPath(parent.path_to_file_parts); + if (parent.path_to_file.empty()) { + parent.full_path = parent.container; + } else { + parent.full_path = parent.container + kSep + parent.path_to_file; + } + return parent; + } + + bool has_parent() const { return !path_to_file.empty(); } + + bool empty() const { return container.empty() && path_to_file.empty(); } + + bool operator==(const AzurePath& other) const { + return container == other.container && path_to_file == other.path_to_file; + } +}; + +template +std::shared_ptr GetObjectMetadata(const ObjectResult& result) { + auto md = std::make_shared(); + auto push = [&](std::string k, const std::string v) { + if (!v.empty()) { + md->Append(std::move(k), v); + } + }; + for (auto prop: result) { + push(prop.first, prop.second); + } + return md; +} + +template +Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, const std::string url) { + if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { + client = std::make_shared(url, options.storage_credentials_provider); + } else if(options.credentials_kind == AzureCredentialsKind::ServicePrincipleCredentials) { + client = std::make_shared(url, options.service_principle_credentials_provider); + } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { + client = std::make_shared(T::CreateFromConnectionString(options.connection_string)); + } else if (options.credentials_kind == AzureCredentialsKind::Sas){ + client = std::make_shared(url + options.sas_token); + } else { + client = std::make_shared(url); + } + return Status::OK(); +} + +template +Status InitPathClient(std::shared_ptr& client, const AzureOptions options, const std::string path, const std::string container, const std::string path_to_file) { + if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { + client = std::make_shared(path, options.storage_credentials_provider); + } else if(options.credentials_kind == AzureCredentialsKind::ServicePrincipleCredentials) { + client = std::make_shared(path, options.service_principle_credentials_provider); + } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { + client = std::make_shared(T::CreateFromConnectionString(options.connection_string, container, path_to_file)); + } else if (options.credentials_kind == AzureCredentialsKind::Sas) { + auto src = internal::RemoveLeadingSlash(path); + auto first_sep = src.find("dfs.core.windows.net"); + std::string p; + if (first_sep != std::string::npos) { + p = std::string(src.substr(0, first_sep)) + "blob" + std::string(src.substr(first_sep + 3)); + client = std::make_shared(p + options.sas_token); + } else { + client = std::make_shared(path); + } + } else { + client = std::make_shared(path); + } + return Status::OK(); +} + +class ObjectInputFile final : public io::RandomAccessFile { + public: + ObjectInputFile(std::shared_ptr pathClient, std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, int64_t size = kNoSize) + : pathClient_(std::move(pathClient)), + fileClient_(std::move(fileClient)), + io_context_(io_context), + path_(path), + content_length_(size) {} + + Status Init() { + if (content_length_ != kNoSize) { + DCHECK_GE(content_length_, 0); + return Status::OK(); + } + try { + auto properties = pathClient_->GetProperties(); + if (properties.Value.IsDirectory) { + return Status::IOError("Invalid file path given"); + } + content_length_ = properties.Value.FileSize; + DCHECK_GE(content_length_, 0); + metadata_ = GetObjectMetadata(properties.Value.Metadata); + return Status::OK(); + } catch (std::exception const& e) { + return Status::IOError("Invalid file path given"); + } + } + + Status CheckClosed() const { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + return Status::OK(); + } + + Status CheckPosition(int64_t position, const char* action) const { + if (position < 0) { + return Status::Invalid("Cannot ", action, " from negative position"); + } + if (position > content_length_) { + return Status::IOError("Cannot ", action, " past end of file"); + } + return Status::OK(); + } + + // RandomAccessFile APIs + + Result> ReadMetadata() override { + return metadata_; + } + + Future> ReadMetadataAsync( + const io::IOContext& io_context) override { + return metadata_; + } + + Status Close() override { + pathClient_ = nullptr; + fileClient_ = nullptr; + closed_ = true; + return Status::OK(); + } + + bool closed() const override { return closed_; } + + Result Tell() const override { + RETURN_NOT_OK(CheckClosed()); + return pos_; + } + + Result GetSize() override { + RETURN_NOT_OK(CheckClosed()); + return content_length_; + } + + Status Seek(int64_t position) override { + RETURN_NOT_OK(CheckClosed()); + RETURN_NOT_OK(CheckPosition(position, "seek")); + + pos_ = position; + return Status::OK(); + } + + Result ReadAt(int64_t position, int64_t nbytes, void* out) override { + RETURN_NOT_OK(CheckClosed()); + RETURN_NOT_OK(CheckPosition(position, "read")); + + nbytes = std::min(nbytes, content_length_ - position); + if (nbytes == 0) { + return 0; + } + + // Read the desired range of bytes + Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; + Azure::Core::Http::HttpRange range; + range.Offset = position; + range.Length = nbytes; + downloadOptions.Range = Azure::Nullable(range); + auto result = + fileClient_->DownloadTo(reinterpret_cast(out), nbytes, downloadOptions) + .Value; + AZURE_ASSERT(result.ContentRange.Length.HasValue()); + return result.ContentRange.Length.Value(); + } + + Result> ReadAt(int64_t position, int64_t nbytes) override { + RETURN_NOT_OK(CheckClosed()); + RETURN_NOT_OK(CheckPosition(position, "read")); + + // No need to allocate more than the remaining number of bytes + nbytes = std::min(nbytes, content_length_ - position); + + ARROW_ASSIGN_OR_RAISE(auto buf, AllocateResizableBuffer(nbytes, io_context_.pool())); + if (nbytes > 0) { + ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, + ReadAt(position, nbytes, buf->mutable_data())); + DCHECK_LE(bytes_read, nbytes); + RETURN_NOT_OK(buf->Resize(bytes_read)); + } + return std::move(buf); + } + + Result Read(int64_t nbytes, void* out) override { + ARROW_ASSIGN_OR_RAISE(int64_t bytes_read, ReadAt(pos_, nbytes, out)); + pos_ += bytes_read; + return bytes_read; + } + + Result> Read(int64_t nbytes) override { + ARROW_ASSIGN_OR_RAISE(auto buffer, ReadAt(pos_, nbytes)); + pos_ += buffer->size(); + return std::move(buffer); + } + + protected: + std::shared_ptr pathClient_; + std::shared_ptr fileClient_; + const io::IOContext io_context_; + AzurePath path_; + + bool closed_ = false; + int64_t pos_ = 0; + int64_t content_length_ = kNoSize; + std::shared_ptr metadata_; +}; + +class ObjectOutputStream final : public io::OutputStream { + public: + ObjectOutputStream(std::shared_ptr pathClient, std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) + : pathClient_(std::move(pathClient)), + fileClient_(std::move(fileClient)), + io_context_(io_context), + path_(path), + metadata_(metadata){} + + ~ObjectOutputStream() override { + // For compliance with the rest of the IO stack, Close rather than Abort, + // even though it may be more expensive. + io::internal::CloseFromDestructor(this); + } + + Status Init() { + closed_ = false; + if (content_length_ != kNoSize) { + DCHECK_GE(content_length_, 0); + return Status::OK(); + } + try{ + auto properties = pathClient_->GetProperties(); + if (properties.Value.IsDirectory) { + return Status::IOError("Invalid file path given"); + } + content_length_ = properties.Value.FileSize; + DCHECK_GE(content_length_, 0); + } catch(std::exception const& e) { + //new file + std::string s = ""; + fileClient_->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + content_length_ = 0; + } + return Status::OK(); + } + + Status Abort() override { + if (closed_) { + return Status::OK(); + } + pathClient_ = nullptr; + fileClient_ = nullptr; + closed_ = true; + return Status::OK(); + } + + // OutputStream interface + + Status Close() override { + if (closed_) { + return Status::OK(); + } + pathClient_ = nullptr; + fileClient_ = nullptr; + closed_ = true; + return Status::OK(); + } + + bool closed() const override { return closed_; } + + Result Tell() const override { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + return pos_; + } + + Status Write(const std::shared_ptr& buffer) override { + return DoWrite(buffer->data(), buffer->size(), buffer); + } + + Status Write(const void* data, int64_t nbytes) override { + return DoWrite(data, nbytes); + } + + Status DoWrite(const void* data, int64_t nbytes, + std::shared_ptr owned_buffer = nullptr) { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + auto result = fileClient_->UploadFrom(const_cast(reinterpret_cast(data)), nbytes).Value; + pos_ += nbytes; + return Status::OK(); + } + + Status Flush() override { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + fileClient_->Flush(content_length_); + return Status::OK(); + } + + protected: + std::shared_ptr pathClient_; + std::shared_ptr fileClient_; + const io::IOContext io_context_; + const AzurePath path_; + + bool closed_ = true; + int64_t pos_ = 0; + int64_t content_length_ = kNoSize; + std::shared_ptr metadata_; +}; + +class ObjectAppendStream final : public io::OutputStream { + public: + ObjectAppendStream(std::shared_ptr pathClient, std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) + : pathClient_(std::move(pathClient)), + fileClient_(std::move(fileClient)), + io_context_(io_context), + path_(path), + metadata_(metadata){} + + ~ObjectAppendStream() override { + // For compliance with the rest of the IO stack, Close rather than Abort, + // even though it may be more expensive. + io::internal::CloseFromDestructor(this); + } + + Status Init() { + closed_ = false; + if (content_length_ != kNoSize) { + DCHECK_GE(content_length_, 0); + return Status::OK(); + } + try{ + auto properties = pathClient_->GetProperties(); + if (properties.Value.IsDirectory) { + return Status::IOError("Invalid file path given"); + } + content_length_ = properties.Value.FileSize; + DCHECK_GE(content_length_, 0); + return Status::OK(); + } catch(std::exception const& e) { + return Status::IOError("Invalid file path given"); + } + } + + Status Abort() override { + if (closed_) { + return Status::OK(); + } + pathClient_ = nullptr; + fileClient_ = nullptr; + closed_ = true; + return Status::OK(); + } + + // OutputStream interface + + Status Close() override { + if (closed_) { + return Status::OK(); + } + pathClient_ = nullptr; + fileClient_ = nullptr; + closed_ = true; + return Status::OK(); + } + + bool closed() const override { return closed_; } + + Result Tell() const override { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + return pos_; + } + + Status Write(const std::shared_ptr& buffer) override { + return DoAppend(buffer->data(), buffer->size(), buffer); + } + + Status Write(const void* data, int64_t nbytes) override { + return DoAppend(data, nbytes); + } + + Status DoAppend(const void* data, int64_t nbytes, + std::shared_ptr owned_buffer = nullptr) { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + auto content = Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes); + auto result = fileClient_->Append(content, 0); + return Status::OK(); + } + + Status Flush() override { + if (closed_) { + return Status::Invalid("Operation on closed stream"); + } + fileClient_->Flush(content_length_); + return Status::OK(); + } + + protected: + std::shared_ptr pathClient_; + std::shared_ptr fileClient_; + const io::IOContext io_context_; + const AzurePath path_; + + bool closed_ = true; + int64_t pos_ = 0; + int64_t content_length_ = kNoSize; + const std::shared_ptr metadata_; +}; + +TimePoint ToTimePoint(int secs) { + std::chrono::nanoseconds ns_count(static_cast(secs) * 1000000000); + return TimePoint(std::chrono::duration_cast(ns_count)); +} + +void FileObjectToInfo(const Azure::Storage::Files::DataLake::Models::PathProperties& properties, FileInfo* info) { + info->set_type(FileType::File); + info->set_size(static_cast(properties.FileSize)); + info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(properties.LastModified))); +} + +void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, const Azure::DateTime dt, FileInfo* info) { + info->set_type(type); + info->set_size(size); + info->set_path(path); + info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); +} + +} // namespace + +// ----------------------------------------------------------------------- +// Azure filesystem implementation + +class AzureBlobFileSystem::Impl : public std::enable_shared_from_this { + public: + io::IOContext io_context_; + std::shared_ptr gen1Client_; + std::shared_ptr gen2Client_; + std::string dfs_endpoint_url; + std::string blob_endpoint_url; + bool isHierarchicalNamespaceEnabled; + + explicit Impl(AzureOptions options, io::IOContext io_context) + : io_context_(io_context), options_(std::move(options)) {} + + Status Init() { + dfs_endpoint_url = options_.account_dfs_url; + blob_endpoint_url = options_.account_blob_url; + InitServiceClient(gen1Client_, options_, blob_endpoint_url); + InitServiceClient(gen2Client_, options_, dfs_endpoint_url); + isHierarchicalNamespaceEnabled = gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + return Status::OK(); + } + + const AzureOptions& options() const { return options_; } + + //Create a container. Successful if container already exists. + Status CreateContainer(const std::string& container) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + fileSystemClient.CreateIfNotExists(); + return Status::OK(); + } + + //Tests to see if a container exists + Result ContainerExists(const std::string& container) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + try { + auto properties = fileSystemClient.GetProperties(); + return true; + } catch (std::exception const& e){ + return false; + } + } + + Result DirExists(const std::string& s) { + std::string uri = s; + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, uri, path.container, path.path_to_file); + try { + auto properties = pathClient_->GetProperties(); + return properties.Value.IsDirectory; + } catch (std::exception const& e) { + return false; + } + } + + Result FileExists(const std::string& s) { + std::string uri = s; + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, uri, path.container, path.path_to_file); + try { + auto properties = pathClient_->GetProperties(); + return !properties.Value.IsDirectory; + } catch (std::exception const& e) { + return false; + } + } + + Status CreateEmptyDir(const std::string& container, const std::vector& path) { + auto directoryClient = gen2Client_->GetFileSystemClient(container).GetDirectoryClient(path.front()); + std::vector::const_iterator it = path.begin(); + std::advance(it, 1); + while (it != path.end()) { + directoryClient = directoryClient.GetSubdirectoryClient(*it); + ++it; + } + directoryClient.CreateIfNotExists(); + return Status::OK(); + } + + Status DeleteContainer(const std::string& container) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + fileSystemClient.DeleteIfExists(); + return Status::OK(); + } + + Status DeleteDir(const std::string& container, const std::vector& path) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + std::vector::const_iterator it = path.begin(); + std::advance(it, 1); + while (it != path.end()) { + directoryClient = directoryClient.GetSubdirectoryClient(*it); + ++it; + } + if (FileExists(directoryClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete directory, Invalid Directory Path"); + } + if (!DirExists(directoryClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete directory, Invalid Directory Path"); + } + directoryClient.DeleteRecursiveIfExists(); + return Status::OK(); + } + + Status DeleteFile(const std::string& container, const std::vector& path) { + if (path.empty()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + if (!isHierarchicalNamespaceEnabled) { + if (path.size() > 1) { + return Status::IOError("Cannot delete File, Invalid File Path, hierarchical namespace not enabled"); + } + auto blobClient = gen1Client_->GetBlobContainerClient(container).GetBlobClient(path.front()); + if (!FileExists(blobClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + blobClient.DeleteIfExists(); + return Status::OK(); + } + auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + if (path.size() == 1) { + auto fileClient = fileSystemClient.GetFileClient(path.front()); + if (DirExists(fileClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + if (!FileExists(fileClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + fileClient.DeleteIfExists(); + return Status::OK(); + } + std::string file_name = path.back(); + auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + std::vector::const_iterator it = path.begin(); + std::advance(it, 1); + while (it != (path.end()-1)) { + directoryClient = directoryClient.GetSubdirectoryClient(*it); + ++it; + } + auto fileClient = directoryClient.GetFileClient(file_name); + if (DirExists(fileClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + if (!FileExists(fileClient.GetUrl()).ValueOrDie()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + fileClient.DeleteIfExists(); + return Status::OK(); + } + + Status Move(const std::string& src, const std::string& dest) { + ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); + ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); + + if (!isHierarchicalNamespaceEnabled) { + return Status::IOError("Cannot move, Hierarchical namespace not enabled"); + } + if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || dest_path.path_to_file.empty()) { + return Status::IOError("Invalid path provided"); + } + if (src_path == dest_path) { + return Status::OK(); + } + if (FileExists(dfs_endpoint_url + src_path.full_path).ValueOrDie()) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(src_path.container); + auto path = src_path.path_to_file_parts; + if (path.size() == 1) { + try { + fileSystemClient.RenameFile(path.front(), dest_path.path_to_file); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + return Status::OK(); + } + auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + std::vector::const_iterator it = path.begin(); + std::advance(it, 1); + while (it != path.end()) { + if ((it+1) == path.end()) { + break; + } + directoryClient = directoryClient.GetSubdirectoryClient(*it); + ++it; + } + try { + directoryClient.RenameFile(it->data(), dest_path.path_to_file); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else if (DirExists(dfs_endpoint_url + src_path.full_path).ValueOrDie()) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(src_path.container); + auto path = src_path.path_to_file_parts; + if (path.size() == 1) { + try { + fileSystemClient.RenameDirectory(path.front(), dest_path.path_to_file); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + return Status::OK(); + } + auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + std::vector::const_iterator it = path.begin(); + std::advance(it, 1); + while (it != path.end()) { + if ((it+1) == path.end()) { + break; + } + directoryClient = directoryClient.GetSubdirectoryClient(*it); + ++it; + } + try { + directoryClient.RenameSubdirectory(it->data(), dest_path.path_to_file); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else { + return Status::IOError("Invalid path provided"); + } + return Status::OK(); + } + + Status CopyFile(const std::string& src, const std::string& dest) { + ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); + ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); + + if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || dest_path.path_to_file.empty()) { + return Status::IOError("Cannot copy file, file doesn't exist at src"); + } + + if (!(FileExists(dfs_endpoint_url + src_path.full_path)).ValueOrDie()) { + return Status::IOError("Cannot copy file, file doesn't exist at src"); + } + + if (DirExists(dfs_endpoint_url + dest_path.full_path).ValueOrDie()) { + return Status::IOError("Cannot copy file, Invalid destination path"); + } + + if (!isHierarchicalNamespaceEnabled) { + if (src_path.path_to_file_parts.size() > 1 || dest_path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + if (dest_path.empty() || dest_path.path_to_file_parts.empty()) { + return Status::IOError("Invalid path provided at destination"); + } + if (src_path == dest_path) { + return Status::OK(); + } + auto containerClient = gen1Client_->GetBlobContainerClient(dest_path.container); + auto fileClient = containerClient.GetBlobClient(dest_path.path_to_file); + try { + auto response = fileClient.StartCopyFromUri(blob_endpoint_url + src); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + return Status::OK(); + } + + if (dest_path.has_parent()) { + AzurePath parent_path = dest_path.parent(); + if (parent_path.path_to_file.empty()) { + if (!ContainerExists(parent_path.container).ValueOrDie()) { + return Status::IOError("Cannot copy file '", src_path.full_path, + "': parent directory of destination does not exist"); + } + } else { + auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot copy file '", src_path.full_path, + "': parent directory of destination does not exist"); + } + } + } + if (src_path == dest_path) { + return Status::OK(); + } + auto containerClient = gen1Client_->GetBlobContainerClient(dest_path.container); + auto fileClient = containerClient.GetBlobClient(dest_path.path_to_file); + try { + if (options_.credentials_kind == AzureCredentialsKind::Sas) { + fileClient.StartCopyFromUri(blob_endpoint_url + src + options_.sas_token); + } else { + fileClient.StartCopyFromUri(blob_endpoint_url + src); + } + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + return Status::OK(); + } + + Status ListPaths(const std::string& container, const std::string& path, + std::vector* childrenDirs, std::vector* childrenFiles, const bool allow_not_found = false) { + if (!isHierarchicalNamespaceEnabled) { + try { + auto paths = gen1Client_->GetBlobContainerClient(container).ListBlobs(); + for (auto p : paths.Blobs) { + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + childrenFiles->push_back(container+"/"+p.Name); + } + } catch (std::exception const& e) { + if (!allow_not_found) { + return Status::IOError("Path does not exists"); + } + } + return Status::OK(); + } + if (path.empty()) { + try { + auto paths = gen2Client_->GetFileSystemClient(container).ListPaths(false); + for (auto p : paths.Paths) { + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + if (pathClient_->GetProperties().Value.IsDirectory) { + childrenDirs->push_back(container+"/"+p.Name); + } else { + childrenFiles->push_back(container+"/"+p.Name); + } + } + } catch (std::exception const& e) { + if (!allow_not_found) { + return Status::IOError("Path does not exists"); + } + } + return Status::OK(); + } + std::vector dirs = internal::SplitAbstractPath(path); + try { + Azure::Storage::Files::DataLake::DataLakeDirectoryClient dirClient = gen2Client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); + for (auto dir=dirs.begin()+1; dir pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + if (pathClient_->GetProperties().Value.IsDirectory) { + childrenDirs->push_back(container+"/"+p.Name); + } else { + childrenFiles->push_back(container+"/"+p.Name); + } + } + } catch (std::exception const& e) { + if (!allow_not_found) { + return Status::IOError("Path does not exists"); + } + } + return Status::OK(); + } + + Status Walk(const FileSelector& select, const std::string& container, + const std::string& path, int nesting_depth, std::vector* out) { + + std::vector childrenDirs; + std::vector childrenFiles; + + Status st = ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); + if(!st.ok()) { + return st; + } + + for (const auto& childFile : childrenFiles) { + FileInfo info; + // std::string url = gen2Client_->GetUrl(); + Azure::Storage::Files::DataLake::Models::PathProperties properties; + GetProperties(dfs_endpoint_url+childFile, &properties); + PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, properties.LastModified, &info); + out->push_back(std::move(info)); + } + for (const auto& childDir : childrenDirs) { + FileInfo info; + // std::string url = gen2Client_->GetUrl(); + Azure::Storage::Files::DataLake::Models::PathProperties properties; + GetProperties(dfs_endpoint_url+childDir, &properties); + PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, &info); + out->push_back(std::move(info)); + if (select.recursive && nesting_depth < select.max_recursion) { + const auto src = internal::RemoveTrailingSlash(childDir); + auto first_sep = src.find_first_of("/"); + std::string s = std::string(src.substr(first_sep + 1)); + RETURN_NOT_OK(Walk(select, container, s, nesting_depth + 1, out)); + } + } + return Status::OK(); + } + + Status GetProperties(const std::string& s, Azure::Storage::Files::DataLake::Models::PathProperties* properties) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, s, path.container, path.path_to_file); + if (path.path_to_file.empty()) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(path.container); + auto props = fileSystemClient.GetProperties().Value; + properties->LastModified = props.LastModified; + properties->Metadata = props.Metadata; + properties->ETag = props.ETag; + properties->FileSize = -1; + return Status::OK(); + } + auto props = pathClient_->GetProperties().Value; + properties->FileSize = props.FileSize; + properties->LastModified = props.LastModified; + properties->Metadata = props.Metadata; + properties->ETag = props.ETag; + return Status::OK(); + } + + Status DeleteDirContents(const std::string& container, const std::string& path, const std::vector& path_to_file_parts) { + + std::vector childrenDirs; + std::vector childrenFiles; + + Status st = ListPaths(container, path, &childrenDirs, &childrenFiles); + if(!st.ok()) { + return st; + } + for (const auto& childFile : childrenFiles) { + ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); + DeleteFile(filePath.container, filePath.path_to_file_parts); + } + for (const auto& childDir : childrenDirs) { + ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); + DeleteDir(dirPath.container, dirPath.path_to_file_parts); + } + return Status::OK(); + } + + + Result> ListContainers() { + auto outcome = gen2Client_->ListFileSystems(); + std::vector containers; + for (auto container: outcome.FileSystems) { + containers.push_back(container.Name); + } + return containers; + } + + Result> OpenInputFile(const std::string& s, AzureBlobFileSystem* fs) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty()) { + return Status::IOError("Invalid path provided"); + } + if (!isHierarchicalNamespaceEnabled) { + if (path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + } + if (!(FileExists(dfs_endpoint_url + path.full_path)).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + + std::shared_ptr fileClient_; + InitPathClient(fileClient_, options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + + auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path); + RETURN_NOT_OK(ptr->Init()); + return ptr; + } + + Result> OpenOutputStream(const std::string& s, const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty() || path.path_to_file.empty()) { + return Status::IOError("Invalid path provided"); + } + std::string endpoint_url = dfs_endpoint_url; + if (!isHierarchicalNamespaceEnabled) { + if (path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + endpoint_url = blob_endpoint_url; + } + if (DirExists(dfs_endpoint_url + path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, endpoint_url + path.full_path, path.container, path.path_to_file); + + std::shared_ptr fileClient_; + InitPathClient(fileClient_, options_, endpoint_url + path.full_path, path.container, path.path_to_file); + + if (path.has_parent()) { + AzurePath parent_path = path.parent(); + if (parent_path.path_to_file.empty()) { + if (!ContainerExists(parent_path.container).ValueOrDie()) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } else { + auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } + } + auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, metadata); + RETURN_NOT_OK(ptr->Init()); + return ptr; + } + + Result> OpenAppendStream(const std::string& s, const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (!isHierarchicalNamespaceEnabled) { + if (path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + } + + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url+s, path.container, path.path_to_file); + + std::shared_ptr fileClient_; + InitPathClient(fileClient_, options_, dfs_endpoint_url+s, path.container, path.path_to_file); + + auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, metadata); + RETURN_NOT_OK(ptr->Init()); + return ptr; + } + + Result> OpenInputFile(const FileInfo& info, + AzureBlobFileSystem* fs) { + if (info.type() == FileType::NotFound) { + return ::arrow::fs::internal::PathNotFound(info.path()); + } + if (info.type() != FileType::File && info.type() != FileType::Unknown) { + return ::arrow::fs::internal::NotAFile(info.path()); + } + + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(info.path())); + + if (!isHierarchicalNamespaceEnabled) { + if (path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + } + if (!(FileExists(dfs_endpoint_url + info.path())).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + std::shared_ptr pathClient_; + InitPathClient(pathClient_, options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + + std::shared_ptr fileClient_; + InitPathClient(fileClient_, options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + + auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, info.size()); + RETURN_NOT_OK(ptr->Init()); + return ptr; + } + + protected: + AzureOptions options_; +}; + +AzureBlobFileSystem::AzureBlobFileSystem(const AzureOptions& options, const io::IOContext& io_context) + : FileSystem(io_context), impl_(std::make_shared(options, io_context)) { + default_async_is_sync_ = false; +} + +AzureBlobFileSystem::~AzureBlobFileSystem() {} + +Result> AzureBlobFileSystem::Make( + const AzureOptions& options, const io::IOContext& io_context) { + std::shared_ptr ptr(new AzureBlobFileSystem(options, io_context)); + RETURN_NOT_OK(ptr->impl_->Init()); + return ptr; +} + +bool AzureBlobFileSystem::Equals(const FileSystem& other) const { + if (this == &other) { + return true; + } + if (other.type_name() != type_name()) { + return false; + } + const auto& azurefs = ::arrow::internal::checked_cast(other); + return options().Equals(azurefs.options()); +} + +AzureOptions AzureBlobFileSystem::options() const { return impl_->options(); } + +Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + FileInfo info; + info.set_path(s); + + if (!impl_->isHierarchicalNamespaceEnabled) { + if (path.path_to_file_parts.size() > 1) { + info.set_type(FileType::NotFound); + return info; + } + } + + if (path.empty()) { + // It's the root path "" + info.set_type(FileType::Directory); + return info; + } else if (path.path_to_file.empty()) { + // It's a container + ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); + if (!container_exists) { + info.set_type(FileType::NotFound); + return info; + } + info.set_type(FileType::Directory); + return info; + } else { + // It's an object + ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)); + if (file_exists) { + // "File" object found + Azure::Storage::Files::DataLake::Models::PathProperties properties; + impl_->GetProperties(impl_->dfs_endpoint_url + path.full_path, &properties); + FileObjectToInfo(properties, &info); + return info; + } + // Not found => perhaps it's a "directory" + auto is_dir = impl_->DirExists(impl_->dfs_endpoint_url + path.full_path); + if (is_dir.ValueOrDie()) { + info.set_type(FileType::Directory); + } else { + info.set_type(FileType::NotFound); + } + return info; + } +} + +Result AzureBlobFileSystem::GetFileInfo(const FileSelector& select) { + ARROW_ASSIGN_OR_RAISE(auto base_path, AzurePath::FromString(select.base_dir)); + + FileInfoVector results; + + if (base_path.empty()) { + // List all containers + ARROW_ASSIGN_OR_RAISE(auto containers, impl_->ListContainers()); + for (const auto& container : containers) { + FileInfo info; + // std::string url = impl_->gen2Client_->GetUrl(); + Azure::Storage::Files::DataLake::Models::PathProperties properties; + impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties); + PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, &info); + results.push_back(std::move(info)); + if (select.recursive) { + RETURN_NOT_OK(impl_->Walk(select, container, "", 0, &results)); + } + } + return results; + } + + if (!impl_->isHierarchicalNamespaceEnabled) { + if (base_path.path_to_file_parts.size() > 1) { + if (!select.allow_not_found) { + return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + } + return results; + } + } + + if (base_path.path_to_file.empty() && !(impl_->ContainerExists(base_path.container).ValueOrDie())) { + if (!select.allow_not_found) { + return Status::IOError("Invalid path provided"); + } + return results; + } + + if (impl_->FileExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + + if (!(base_path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie())) { + if (!select.allow_not_found) { + return Status::IOError("Invalid path provided"); + } + return results; + } + + // Nominal case -> walk a single container + RETURN_NOT_OK(impl_->Walk(select, base_path.container, base_path.path_to_file, 0, &results)); + return results; +} + +Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty()) { + return Status::IOError("Cannot create directory, root path given"); + } + if ((impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)).ValueOrDie()) { + return Status::IOError("Cannot create directory, file exists at path"); + } + if (path.path_to_file.empty()) { + // Create container + return impl_->CreateContainer(path.container); + } + //Hierarchical namespace not enabled type storage accounts + if (!impl_->isHierarchicalNamespaceEnabled) { + if (!path.path_to_file.empty()) { + return Status::IOError("Cannot create directory, storage account doesn't have hierarchical namespace enabled"); + } + } + if (recursive) { + // Ensure container exists + ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); + if (!container_exists) { + RETURN_NOT_OK(impl_->CreateContainer(path.container)); + } + std::vector parent_path_to_file; + + for (const auto& part : path.path_to_file_parts) { + parent_path_to_file.push_back(part); + RETURN_NOT_OK(impl_->CreateEmptyDir(path.container, parent_path_to_file)); + } + return Status::OK(); + } else { + // Check parent dir exists + if (path.has_parent()) { + AzurePath parent_path = path.parent(); + if (parent_path.path_to_file.empty()) { + auto exists = impl_->ContainerExists(parent_path.container); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot create directory '", path.full_path, + "': parent directory does not exist"); + } + } else { + auto exists = impl_->DirExists(impl_->dfs_endpoint_url + parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot create directory '", path.full_path, + "': parent directory does not exist"); + } + } + } + return impl_->CreateEmptyDir(path.container, path.path_to_file_parts); + } +} + +Status AzureBlobFileSystem::DeleteDir(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + if (path.empty()) { + return Status::NotImplemented("Cannot delete all Azure Containers"); + } + if (path.path_to_file.empty()) { + return impl_->DeleteContainer(path.container); + } + if ((impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)).ValueOrDie()) { + return Status::IOError("Cannot delete directory, file exists at path"); + } + + //Hierarchical namespace not enabled type storage accounts + if (!impl_->isHierarchicalNamespaceEnabled) { + if (!path.path_to_file.empty()) { + return Status::IOError("Cannot delete directory, storage account doesn't have hierarchical namespace enabled"); + } + } + return impl_->DeleteDir(path.container, path.path_to_file_parts); +} + +Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty()) { + return Status::IOError("Invalid path provided"); + } + + if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container).ValueOrDie())) { + return Status::IOError("Invalid path provided1"); + } + + if (impl_->FileExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided2"); + } + + if (!(path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { + return Status::IOError("Invalid path provided3"); + } + + return impl_->DeleteDirContents(path.container, path.path_to_file, path.path_to_file_parts); +} + +Status AzureBlobFileSystem::DeleteRootDirContents() { + return Status::NotImplemented("Cannot delete all Azure Containers"); +} + +Status AzureBlobFileSystem::DeleteFile(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + return impl_->DeleteFile(path.container, path.path_to_file_parts); +} + +Status AzureBlobFileSystem::Move(const std::string& src, const std::string& dest) { + return impl_->Move(src, dest); +} + +Status AzureBlobFileSystem::CopyFile(const std::string& src, const std::string& dest) { + return impl_->CopyFile(src, dest); +} + +Result> AzureBlobFileSystem::OpenInputStream( + const std::string& s) { + return impl_->OpenInputFile(s, this); +} + +Result> AzureBlobFileSystem::OpenInputStream( + const FileInfo& info) { + return impl_->OpenInputFile(info, this); +} + +Result> AzureBlobFileSystem::OpenInputFile( + const std::string& s) { + return impl_->OpenInputFile(s, this); +} + +Result> AzureBlobFileSystem::OpenInputFile( + const FileInfo& info) { + return impl_->OpenInputFile(info, this); +} + +Result> AzureBlobFileSystem::OpenOutputStream( + const std::string& path, const std::shared_ptr& metadata) { + return impl_->OpenOutputStream(path, metadata, this); +} + +Result> AzureBlobFileSystem::OpenAppendStream( + const std::string& path, const std::shared_ptr& metadata) { + return impl_->OpenAppendStream(path, metadata, this); +} +} +} \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/azure/azurefs.h b/cpp/src/arrow/filesystem/azure/azurefs.h new file mode 100644 index 0000000000000..2aa3b66fbf705 --- /dev/null +++ b/cpp/src/arrow/filesystem/azure/azurefs.h @@ -0,0 +1,157 @@ +// 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 +#include +#include +#include +#include + +#include "arrow/filesystem/filesystem.h" +#include "arrow/util/macros.h" +#include "arrow/util/uri.h" + +namespace arrow { +namespace fs { + +enum class AzureCredentialsKind : int8_t { + /// Anonymous access (no credentials used), public + Anonymous, + /// Use explicitly-provided access key pair + StorageCredentials, + /// + ServicePrincipleCredentials, + /// Use Sas Token to authenticate + Sas, + /// Use Connection String + ConnectionString +}; + +/// Options for the AzureFileSystem implementation. +struct ARROW_EXPORT AzureOptions{ + std::string scheme; + std::string account_dfs_url; + std::string account_blob_url; + AzureCredentialsKind credentials_kind = AzureCredentialsKind::Anonymous; + + std::string sas_token; + std::string connection_string; + std::shared_ptr storage_credentials_provider; + std::shared_ptr service_principle_credentials_provider; + + AzureOptions(); + + void ConfigureAnonymousCredentials(const std::string& account_name); + + void ConfigureAccountKeyCredentials(const std::string& account_name, + const std::string& account_key); + + void ConfigureConnectionStringCredentials(const std::string& connection_string); + + void ConfigureServicePrincipleCredentials(const std::string& account_name, + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret); + + void ConfigureSasCredentials(const std::string& sas_token); + + bool Equals(const AzureOptions& other) const; + + static AzureOptions FromAnonymous(const std::string account_name); + + static AzureOptions FromAccountKey(const std::string& account_name, + const std::string& account_key); + + static AzureOptions FromConnectionString(const std::string& connection_string); + + static AzureOptions FromServicePrincipleCredential(const std::string& account_name, + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret); + + static AzureOptions FromSas(const std::string& uri); + + static Result FromUri(const ::arrow::internal::Uri& uri, + std::string* out_path = NULLPTR); + static Result FromUri(const std::string& uri, + std::string* out_path = NULLPTR); +}; + +class ARROW_EXPORT AzureBlobFileSystem : public FileSystem { + public: + ~AzureBlobFileSystem() override; + + std::string type_name() const override { return "abfs"; } + + /// Return the original Azure options when constructing the filesystem + AzureOptions options() const; + + bool Equals(const FileSystem& other) const override; + + /// \cond FALSE + using FileSystem::GetFileInfo; + /// \endcond + Result GetFileInfo(const std::string& path) override; + Result> GetFileInfo(const FileSelector& select) override; + + /// FileInfoGenerator GetFileInfoGenerator(const FileSelector& select) override; + + Status CreateDir(const std::string& path, bool recursive = true) override; + + Status DeleteDir(const std::string& path) override; + Status DeleteDirContents(const std::string& path) override; + Status DeleteRootDirContents() override; + + Status DeleteFile(const std::string& path) override; + + Status Move(const std::string& src, const std::string& dest) override; + + Status CopyFile(const std::string& src, const std::string& dest) override; + + Result> OpenInputStream( + const std::string& path) override; + + Result> OpenInputStream(const FileInfo& info) override; + + Result> OpenInputFile( + const std::string& path) override; + + Result> OpenInputFile( + const FileInfo& info) override; + + Result> OpenOutputStream( + const std::string& path, + const std::shared_ptr& metadata = {}) override; + + Result> OpenAppendStream( + const std::string& path, + const std::shared_ptr& metadata = {}) override; + + static Result> Make( + const AzureOptions& options, const io::IOContext& = io::default_io_context()); + + protected: + explicit AzureBlobFileSystem(const AzureOptions& options, const io::IOContext&); + + class Impl; + std::shared_ptr impl_; +}; + +} // namespace fs +} // namespace arrow \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/azure/azurefs_test.cc b/cpp/src/arrow/filesystem/azure/azurefs_test.cc new file mode 100644 index 0000000000000..dcddd5ed469b7 --- /dev/null +++ b/cpp/src/arrow/filesystem/azure/azurefs_test.cc @@ -0,0 +1,1096 @@ +// 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/azure/azurefs.h" + +#include +#include + +#include +#include +#include + +#include "arrow/util/uri.h" +#include "arrow/filesystem/test_util.h" +#include "arrow/util/key_value_metadata.h" +#include "arrow/util/logging.h" +#include "arrow/testing/gtest_util.h" +#include "arrow/testing/future_util.h" +#include "arrow/testing/util.h" + +namespace arrow { + +using internal::Uri; + +namespace fs { + +class AzureEnvTestMixin{ + public: + static AzureOptions options_; + static std::shared_ptr fs_; + static std::shared_ptr gen2Client_; + static std::shared_ptr gen1Client_; + + AzureEnvTestMixin() {}; + + const std::string& GetAdlsGen2AccountName() { + const static std::string accountName = [&]() -> std::string { + return std::getenv("ADLS_GEN2_ACCOUNT_NAME"); + }(); + return accountName; + } + + const std::string& GetAdlsGen2AccountKey() { + const static std::string accountKey = [&]() -> std::string { + return std::getenv("ADLS_GEN2_ACCOUNT_KEY"); + }(); + return accountKey; + } + + const std::string& GetAdlsGen2ConnectionString() { + const static std::string connectionString = [&]() -> std::string { + return std::getenv("ADLS_GEN2_CONNECTION_STRING"); + }(); + return connectionString; + } + + const std::string& GetAdlsGen2SasUrl() { + const static std::string sasUrl = [&]() -> std::string { + return std::getenv("ADLS_GEN2_SASURL"); + }(); + return sasUrl; + } + + const std::string& GetAadTenantId() { + const static std::string tenantId = [&]() -> std::string { + return std::getenv("AAD_TENANT_ID"); + }(); + return tenantId; + } + + const std::string& GetAadClientId() { + const static std::string clientId = [&]() -> std::string { + return std::getenv("AAD_CLIENT_ID"); + }(); + return clientId; + } + + const std::string& GetAadClientSecret() { + const static std::string clientSecret = [&]() -> std::string { + return std::getenv("AAD_CLIENT_SECRET"); + }(); + return clientSecret; + } + +// private: +// const std::string& AdlsGen2AccountName = std::getenv("ADLS_GEN2_ACCOUNT_NAME"); +// const std::string& AdlsGen2AccountKey = std::getenv("ADLS_GEN2_ACCOUNT_KEY"); +// const std::string& AdlsGen2ConnectionStringValue = std::getenv("ADLS_GEN2_CONNECTION_STRING"); +// const std::string& AdlsGen2SasUrl = std::getenv("ADLS_GEN2_SASURL"); +// const std::string& AadTenantIdValue = std::getenv("AAD_TENANT_ID"); +// const std::string& AadClientIdValue = std::getenv("AAD_CLIENT_ID"); +// const std::string& AadClientSecretValue = std::getenv("AAD_CLIENT_SECRET"); +}; + +AzureOptions AzureEnvTestMixin::options_; +std::shared_ptr AzureEnvTestMixin::fs_; +std::shared_ptr AzureEnvTestMixin::gen2Client_; +std::shared_ptr AzureEnvTestMixin::gen1Client_; + + +class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin{ + public: + + bool isHeirarchialNamespaceEnabled() { + return AzureEnvTestMixin::gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + } + + void MakeFileSystem() { + const std::string& account_key = GetAdlsGen2AccountKey(); + const std::string& account_name = GetAdlsGen2AccountName(); + AzureEnvTestMixin::options_.ConfigureAccountKeyCredentials(account_name, account_key); + auto url = options_.account_dfs_url; + AzureEnvTestMixin::gen2Client_ = std::make_shared(url, options_.storage_credentials_provider); + AzureEnvTestMixin::gen1Client_ = std::make_shared(options_.account_blob_url, options_.storage_credentials_provider); + auto result = AzureBlobFileSystem::Make(options_); + if (!result.ok()) { + ARROW_LOG(INFO) + << "AzureFileSystem::Make failed, err msg is " + << result.status().ToString(); + return; + } + AzureEnvTestMixin::fs_ = *result; + } + + void SetUp() override { + { + auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container"); + fileSystemClient.CreateIfNotExists(); + fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("empty-container"); + fileSystemClient.CreateIfNotExists(); + } + { + if (isHeirarchialNamespaceEnabled()) { + auto directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container").GetDirectoryClient("emptydir"); + directoryClient.CreateIfNotExists(); + directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container").GetDirectoryClient("somedir"); + directoryClient.CreateIfNotExists(); + directoryClient = directoryClient.GetSubdirectoryClient("subdir"); + directoryClient.CreateIfNotExists(); + auto fileClient = directoryClient.GetFileClient("subfile"); + fileClient.CreateIfNotExists(); + std::string s = "sub data"; + fileClient.UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + fileClient = gen2Client_->GetFileSystemClient("container").GetFileClient("somefile"); + fileClient.CreateIfNotExists(); + s = "some data"; + fileClient.UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + } else { + auto fc = std::make_shared(options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); + std::string s = "some data"; + fc->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + } + } + } + + void TearDown() override { + auto containers = AzureEnvTestMixin::gen2Client_->ListFileSystems(); + for(auto c:containers.FileSystems) { + auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient(c.Name); + fileSystemClient.DeleteIfExists(); + } + } +}; + +class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin{ + public: + + void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* client, const std::string& container, + const std::string& path_to_file, const std::string& expected) { + auto pathClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, options_.storage_credentials_provider); + auto size = pathClient_->GetProperties().Value.FileSize; + auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); + Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; + Azure::Core::Http::HttpRange range; + range.Offset = 0; + range.Length = size; + downloadOptions.Range = Azure::Nullable(range); + auto fileClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, options_.storage_credentials_provider); + auto result = fileClient_->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), size, downloadOptions).Value; + buf->get()->Equals(Buffer(const_cast(reinterpret_cast(&expected[0])), expected.size())); + } + +}; + +TEST(TestAzureFSOptions, FromUri) { + AzureOptions options; + Uri uri; + + //Public container + ASSERT_OK(uri.Parse("https://testcontainer.dfs.core.windows.net/")); + ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Anonymous); + ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); + + //Sas Token + ASSERT_OK(uri.Parse("https://testcontainer.blob.core.windows.net/?dummy_sas_token")); + ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); + ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); + ASSERT_EQ(options.sas_token, "?dummy_sas_token"); +} + +TEST_F(TestAzureFileSystem, FromAccountKey) { + AzureOptions options; + options = AzureOptions::FromAccountKey(this->GetAdlsGen2AccountKey(), this->GetAdlsGen2AccountName()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::StorageCredentials); + ASSERT_NE(options.storage_credentials_provider, nullptr); +} + +TEST_F(TestAzureFileSystem, FromConnectionString) { + AzureOptions options; + options = AzureOptions::FromConnectionString(this->GetAdlsGen2ConnectionString()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ConnectionString); + ASSERT_NE(options.connection_string, ""); +} + +TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { + AzureOptions options; + options = AzureOptions::FromServicePrincipleCredential(this->GetAdlsGen2AccountName(), this->GetAadTenantId(), this->GetAadClientId(), this->GetAadClientSecret()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ServicePrincipleCredentials); + ASSERT_NE(options.service_principle_credentials_provider, nullptr); +} + +TEST_F(TestAzureFileSystem, FromSas) { + AzureOptions options; + options = AzureOptions::FromSas(this->GetAdlsGen2SasUrl()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); + ASSERT_NE(options.sas_token, ""); +} + +TEST_F(TestAzureFileSystem, CreateDirBlobStorage) { + // New container + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + ASSERT_OK(fs_->CreateDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::Directory); + + // Existing container + ASSERT_OK(fs_->CreateDir("container")); + AssertFileInfo(fs_.get(), "container", FileType::Directory); + + ASSERT_RAISES(IOError, fs_->CreateDir("")); + + // Existing "file", should fail + ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); + + // directory, false + ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", false)); + + // directory, true + ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", true)); +} + +TEST_F(TestAzureFileSystem, DeleteDirBlobStorage) { + FileSelector select; + select.base_dir = "container4"; + std::vector infos; + + // Container + ASSERT_OK(fs_->DeleteDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + + // Nonexistent Container + ASSERT_OK(fs_->DeleteDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + + // root + ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); + + // C/F + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); + + // C/NF + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile19")); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); +} + +TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { + FileSelector select; + select.base_dir = "container4"; + std::vector infos; + + // Container + ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); + + // Nonexistent Container + ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); + + // root + ASSERT_RAISES(IOError, fs_->DeleteFile("")); + + // C/F + ASSERT_OK(fs_->DeleteFile("container/somefile")); + + // C/NF + ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); + + // C/D/D + ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); +} + +TEST_F(TestAzureFileSystem, GetFileInfoBlobStorage) { + //Containers + AssertFileInfo(fs_.get(), "container", FileType::Directory); + AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); + + AssertFileInfo(fs_.get(), "", FileType::Directory); + + auto res = fs_->OpenOutputStream("container/base.txt"); + res->get()->Write("Changed the data"); + + // "Files" + AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); + AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); + + // "Directories" + AssertFileInfo(fs_.get(), "container/somedir45/subdir", FileType::NotFound); + AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); +} + +TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { + FileSelector select; + std::vector infos; + + // Non-empty container + select.base_dir = "container"; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 1); + + // Nonexistent container + select.base_dir = "nonexistent-container"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; + + // Root dir + select.base_dir = ""; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 2); + + // C/F + select.base_dir = "container/base.txt"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + + // C/ND/D + select.base_dir = "container/ahsh/agsg"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; + + // NC/D + select.base_dir = "nonexistent-container/agshhs"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; +} + +TEST_F(TestAzureFileSystem, MoveBlobStorage) { + ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); + ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "containerqw/ghdj")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("Changed the data"); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); + ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/ahsh/gssjd")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "containerqw/ghdj")); + ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/gshh")); +} + +TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { + ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "container27/hshj")); + + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("Changed the data"); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/sjdj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); +} + +TEST_F(TestAzureFileSystem, OpenInputStreamBlobStorage) { + std::shared_ptr stream; + std::shared_ptr buf; + + ASSERT_RAISES(IOError, fs_->OpenInputStream("container")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container526/somedir")); + + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("some data"); + + // "Files" + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(2)); + AssertBufferEqual(*buf, "so"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, "me da"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, "ta"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, ""); +} + +TEST_F(TestAzureFileSystem, OpenInputFileBlobStorage) { + std::shared_ptr file; + std::shared_ptr buf; + + ASSERT_RAISES(IOError, fs_->OpenInputFile("container")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container526/somedir")); + + // "Files" + ASSERT_OK_AND_ASSIGN(file, fs_->OpenInputFile("container/somefile")); + ASSERT_OK_AND_EQ(9, file->GetSize()); + ASSERT_OK_AND_ASSIGN(buf, file->Read(4)); + AssertBufferEqual(*buf, "some"); + ASSERT_OK_AND_EQ(9, file->GetSize()); + ASSERT_OK_AND_EQ(4, file->Tell()); + + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(2, 5)); + AssertBufferEqual(*buf, "me da"); + ASSERT_OK_AND_EQ(4, file->Tell()); + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(5, 20)); + AssertBufferEqual(*buf, "data"); + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(9, 20)); + AssertBufferEqual(*buf, ""); + + char result[10]; + ASSERT_OK_AND_EQ(5, file->ReadAt(2, 5, &result)); + ASSERT_OK_AND_EQ(4, file->ReadAt(5, 20, &result)); + ASSERT_OK_AND_EQ(0, file->ReadAt(9, 0, &result)); + + // Reading past end of file + ASSERT_RAISES(IOError, file->ReadAt(10, 20)); + + ASSERT_OK(file->Seek(5)); + ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); + AssertBufferEqual(*buf, "da"); + ASSERT_OK(file->Seek(9)); + ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); + AssertBufferEqual(*buf, ""); + // Seeking past end of file + ASSERT_RAISES(IOError, file->Seek(10)); +} + +TEST_F(TestAzureFileSystem, OpenOutputStreamBlobStorage) { + std::shared_ptr stream; + + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container526/somedir")); + + // Create new empty file + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + + // Create new file with 1 small write + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); + ASSERT_OK(stream->Write("some data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); + + // Create new file with 3 small writes + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); + ASSERT_OK(stream->Write("some ")); + ASSERT_OK(stream->Write("")); + ASSERT_OK(stream->Write("new data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); + + // Create new file with some large writes + std::string s1, s2, s3, s4, s5, expected; + s1 = random_string(6000000, /*seed =*/42); // More than the 5 MB minimum part upload + s2 = "xxx"; + s3 = random_string(6000000, 43); + s4 = "zzz"; + s5 = random_string(600000, 44); + expected = s1 + s2 + s3 + s4 + s5; + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile4")); + for (auto input : {s1, s2, s3, s4, s5}) { + ASSERT_OK(stream->Write(input)); + // Clobber source contents. This shouldn't reflect in the data written. + input.front() = 'x'; + input.back() = 'x'; + } + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); + + // Overwrite + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Write("overwritten data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); + + // Overwrite and make empty + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); +} + +TEST_F(TestAzureFileSystem, DeleteDirContentsBlobStorage) { + FileSelector select; + select.base_dir = "container4/newdir"; + std::vector infos; + + // Container + fs_->CreateDir("container4"); + ASSERT_OK(fs_->DeleteDirContents("container4")); + AssertFileInfo(fs_.get(), "container4", FileType::Directory); + + // Nonexistent Container + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + + // root + ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); + + // C/F + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); + AssertFileInfo(fs_.get(), "container/somefile", FileType::File); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3/somedir")); +} + +TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { + // New container + auto op = fs_->options(); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + ASSERT_OK(fs_->CreateDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::Directory); + + // Existing container + ASSERT_OK(fs_->CreateDir("container")); + AssertFileInfo(fs_.get(), "container", FileType::Directory); + + ASSERT_RAISES(IOError, fs_->CreateDir("")); + + // New "directory", true + AssertFileInfo(fs_.get(), "container/newdir", FileType::NotFound); + ASSERT_OK(fs_->CreateDir("container/newdir", true)); + AssertFileInfo(fs_.get(), "container/newdir", FileType::Directory); + + // New "directory", false + AssertFileInfo(fs_.get(), "container/newdir1", FileType::NotFound); + ASSERT_OK(fs_->CreateDir("container/newdir1", false)); + AssertFileInfo(fs_.get(), "container/newdir1", FileType::Directory); + + // Existing "directory", true + AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); + ASSERT_OK(fs_->CreateDir("container/somedir", true)); + AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); + + // Existing "directory", false + AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); + ASSERT_OK(fs_->CreateDir("container/somedir", false)); + AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); + + // Existing "file", should fail + ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); + + //C/D/D + AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); + ASSERT_OK(fs_->CreateDir("container/somedir/subdir")); + AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); + + auto res = fs_->OpenOutputStream("container/somedir/base.txt"); + res->get()->Write("Changed the data"); + + //C/D/F + AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); + ASSERT_RAISES(IOError, fs_->CreateDir("container/somedir/base.txt")); + AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); + + // New "directory",Parent dir not exists, false + ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", false)); + + // New "directory",Parent dir not exists, true + ASSERT_OK(fs_->CreateDir("container/newdir/newsub/newsubsub", true)); + AssertFileInfo(fs_.get(), "container/newdir/newsub", FileType::Directory); + AssertFileInfo(fs_.get(), "container/newdir/newsub/newsubsub", FileType::Directory); + + // New "directory",Container not exists, false + ASSERT_RAISES(IOError, fs_->CreateDir("container4/newdir", false)); + + // New "directory",Container not exists, true + ASSERT_OK(fs_->CreateDir("container4/newdir", true)); + AssertFileInfo(fs_.get(), "container4", FileType::Directory); + AssertFileInfo(fs_.get(), "container4/newdir", FileType::Directory); +} + +TEST_F(TestAzureFileSystem, DeleteDirAdlsGen2) { + FileSelector select; + select.base_dir = "container4"; + std::vector infos; + + // Container + ASSERT_OK(fs_->DeleteDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + + // Nonexistent Container + ASSERT_OK(fs_->DeleteDir("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + + // root + ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); + + // C/D + ASSERT_OK(fs_->DeleteDir("container4/newdir")); + // ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + // ASSERT_EQ(infos.size(), 0); + + // C/ND + AssertFileInfo(fs_.get(), "container4/newdir", FileType::NotFound); + ASSERT_RAISES(IOError, fs_->DeleteDir("container4/newdir")); + // ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + // ASSERT_EQ(infos.size(), 0); + + // C/F + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); + + // C/D/D + ASSERT_OK(fs_->DeleteDir("container/newdir/newsub")); + + // C/D/F + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir/base.txt")); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); +} + +TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { + FileSelector select; + select.base_dir = "container4"; + std::vector infos; + + // Container + ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); + + // Nonexistent Container + ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); + + // root + ASSERT_RAISES(IOError, fs_->DeleteFile("")); + + // C/D + ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptyDir")); + + // C/ND + ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptyDir1")); + + // C/F + ASSERT_OK(fs_->DeleteFile("container/somefile")); + + // C/NF + ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); + + // C/D/D + ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); + + auto res = fs_->OpenOutputStream("container/somedir/base.txt"); + res->get()->Write("Changed the data"); + + // C/D/F + AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); + ASSERT_OK(fs_->DeleteFile("container/somedir/base.txt")); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); +} + +TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { + //Containers + AssertFileInfo(fs_.get(), "container", FileType::Directory); + AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); + + AssertFileInfo(fs_.get(), "", FileType::Directory); + + // "Directories" + AssertFileInfo(fs_.get(), "container/emptydir", FileType::Directory); + AssertFileInfo(fs_.get(), "container/emptydir1", FileType::NotFound); + + auto res = fs_->OpenOutputStream("container/base.txt"); + res->get()->Write("Changed the data"); + + // "Files" + AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); + AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); + + // "Directories" + AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); + AssertFileInfo(fs_.get(), "container/somedir/subdir/subfile", FileType::File); + + // "Directories" + AssertFileInfo(fs_.get(), "container/somedir45/subdir", FileType::NotFound); + AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); +} + +TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { + FileSelector select; + std::vector infos; + + // Non-empty container + select.base_dir = "container"; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 5); + + // Nonexistent container + select.base_dir = "nonexistent-container"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; + + // Root dir + select.base_dir = ""; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 3); + + // C/D + select.base_dir = "container/somedir"; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 1); + + // C/ND + select.base_dir = "container/sgsgs"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; + + // C/F + select.base_dir = "container/base.txt"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + + // C/D/D + select.base_dir = "container/somedir/subdir"; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 1); + + // C/F + select.base_dir = "container/somedir/subdir/subfile"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + + // C/ND/D + select.base_dir = "container/ahsh/agsg"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; + + // NC/D + select.base_dir = "nonexistent-container/agshhs"; + ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); + select.allow_not_found = true; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + select.allow_not_found = false; +} + +TEST_F(TestAzureFileSystem, MoveAdlsGen2) { + ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); + fs_->CreateDir("container/newdir/newsub/newsubsub", true); + ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); + ASSERT_OK(fs_->Move("container/emptydir", "container/emptydir1")); + ASSERT_OK(fs_->Move("container/emptydir1", "container/emptydir")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); + ASSERT_OK(fs_->Move("container/emptydir", "containerqw/ghdj")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("Changed the data"); + ASSERT_OK(fs_->Move("container/base.txt", "container/somefile")); + ASSERT_OK(fs_->Move("container/somefile", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/ahsh/gssjd")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "containerqw/ghdj")); + ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/gshh")); +} + +TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { + // "File" + ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "container27/hshj")); + + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("Changed the data"); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somedir/subdir/subfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile")); + fs_->DeleteFile("container/somefile3"); + ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); +} + +TEST_F(TestAzureFileSystem, OpenInputStreamGen2) { + std::shared_ptr stream; + std::shared_ptr buf; + + ASSERT_RAISES(IOError, fs_->OpenInputStream("container")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container526/somedir")); + + auto res = fs_->OpenOutputStream("container/somefile"); + res->get()->Write("some data"); + + // "Files" + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(2)); + AssertBufferEqual(*buf, "so"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, "me da"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, "ta"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); + AssertBufferEqual(*buf, ""); + + res = fs_->OpenOutputStream("container/somedir/subdir/subfile"); + res->get()->Write("sub data"); + + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somedir/subdir/subfile")); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); + AssertBufferEqual(*buf, "sub data"); + ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); + AssertBufferEqual(*buf, ""); + ASSERT_OK(stream->Close()); +} + +TEST_F(TestAzureFileSystem, OpenInputFileGen2) { + std::shared_ptr file; + std::shared_ptr buf; + + ASSERT_RAISES(IOError, fs_->OpenInputFile("container")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container526/somedir")); + + // "Files" + ASSERT_OK_AND_ASSIGN(file, fs_->OpenInputFile("container/somefile")); + ASSERT_OK_AND_EQ(9, file->GetSize()); + ASSERT_OK_AND_ASSIGN(buf, file->Read(4)); + AssertBufferEqual(*buf, "some"); + ASSERT_OK_AND_EQ(9, file->GetSize()); + ASSERT_OK_AND_EQ(4, file->Tell()); + + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(2, 5)); + AssertBufferEqual(*buf, "me da"); + ASSERT_OK_AND_EQ(4, file->Tell()); + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(5, 20)); + AssertBufferEqual(*buf, "data"); + ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(9, 20)); + AssertBufferEqual(*buf, ""); + + char result[10]; + ASSERT_OK_AND_EQ(5, file->ReadAt(2, 5, &result)); + ASSERT_OK_AND_EQ(4, file->ReadAt(5, 20, &result)); + ASSERT_OK_AND_EQ(0, file->ReadAt(9, 0, &result)); + + // Reading past end of file + ASSERT_RAISES(IOError, file->ReadAt(10, 20)); + + ASSERT_OK(file->Seek(5)); + ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); + AssertBufferEqual(*buf, "da"); + ASSERT_OK(file->Seek(9)); + ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); + AssertBufferEqual(*buf, ""); + // Seeking past end of file + ASSERT_RAISES(IOError, file->Seek(10)); +} + +TEST_F(TestAzureFileSystem, OpenOutputStreamGen2) { + std::shared_ptr stream; + + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/shjdj/subdir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container526/somedir")); + + // Create new empty file + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + + // Create new file with 1 small write + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); + ASSERT_OK(stream->Write("some data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); + + // Create new file with 3 small writes + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); + ASSERT_OK(stream->Write("some ")); + ASSERT_OK(stream->Write("")); + ASSERT_OK(stream->Write("new data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); + + // Create new file with some large writes + std::string s1, s2, s3, s4, s5, expected; + s1 = random_string(6000000, /*seed =*/42); // More than the 5 MB minimum part upload + s2 = "xxx"; + s3 = random_string(6000000, 43); + s4 = "zzz"; + s5 = random_string(600000, 44); + expected = s1 + s2 + s3 + s4 + s5; + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile4")); + for (auto input : {s1, s2, s3, s4, s5}) { + ASSERT_OK(stream->Write(input)); + // Clobber source contents. This shouldn't reflect in the data written. + input.front() = 'x'; + input.back() = 'x'; + } + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); + + // Overwrite + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Write("overwritten data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); + + // Overwrite and make empty + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); +} + +TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { + FileSelector select; + select.base_dir = "container4/newdir"; + std::vector infos; + + // Container + ASSERT_OK(fs_->DeleteDirContents("container4")); + AssertFileInfo(fs_.get(), "container4", FileType::Directory); + + // Nonexistent Container + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); + + // root + ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); + + fs_->CreateDir("container4/newdir/subdir", true); + + // C/D + ASSERT_OK(fs_->DeleteDirContents("container4/newdir")); + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + + // C/ND + AssertFileInfo(fs_.get(), "container4/newdir1", FileType::NotFound); + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container4/newdir1")); + + // C/F + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); + AssertFileInfo(fs_.get(), "container/somefile", FileType::File); + + // C/D/D + + ASSERT_OK(fs_->DeleteDirContents("container/somedir/subdir")); + select.base_dir = "container/somedir/subdir"; + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); + + // C/ND/D + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somedir3/base")); + + // NC/D + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3/somedir")); +} + +} // namespace fs +} // namespace arrow + +int main(int argc, char **argv) { + auto env = new arrow::fs::SetupEnvironment(); + env->MakeFileSystem(); + ::testing::AddGlobalTestEnvironment(env); + ::testing::InitGoogleTest(&argc, argv); + if (env->isHeirarchialNamespaceEnabled()) { + ::testing::GTEST_FLAG(filter) = "*From*:*Gen2"; + } else { + ::testing::GTEST_FLAG(filter) = "*From*:*BlobStorage"; + } + return RUN_ALL_TESTS(); +} \ No newline at end of file diff --git a/cpp/vcpkg.json b/cpp/vcpkg.json index 6fadf299bbf21..5c4ef25558655 100644 --- a/cpp/vcpkg.json +++ b/cpp/vcpkg.json @@ -14,6 +14,11 @@ "transfer" ] }, + "azure-core-cpp", + "azure-identity-cpp", + "azure-storage-blobs-cpp", + "azure-storage-common-cpp", + "azure-storage-files-datalake-cpp", "benchmark", "boost-filesystem", "boost-multiprecision", From 1e2d0a3bd7310e1c4b7e32adb10b29e0f146e434 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 21 Apr 2022 09:38:28 +0530 Subject: [PATCH 02/34] ARROW-2034: [C++] Fixed formatting issues --- cpp/src/arrow/filesystem/azure/azurefs.cc | 308 +++++++++++------- cpp/src/arrow/filesystem/azure/azurefs.h | 30 +- .../arrow/filesystem/azure/azurefs_test.cc | 214 +++++++----- 3 files changed, 341 insertions(+), 211 deletions(-) diff --git a/cpp/src/arrow/filesystem/azure/azurefs.cc b/cpp/src/arrow/filesystem/azure/azurefs.cc index 583f70ee3f64c..a06f81c0287fe 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs.cc @@ -85,16 +85,19 @@ void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name credentials_kind = AzureCredentialsKind::Anonymous; } -void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, - const std::string& account_key) { +void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, + const std::string& account_key) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; - storage_credentials_provider = std::make_shared(account_name, account_key); + storage_credentials_provider = std::make_shared(account_name, account_key); credentials_kind = AzureCredentialsKind::StorageCredentials; } -void AzureOptions::ConfigureConnectionStringCredentials(const std::string& connection_string_uri) { - auto account_name = Azure::Storage::_internal::ParseConnectionString(connection_string_uri).AccountName; +void AzureOptions::ConfigureConnectionStringCredentials(const std::string& + connection_string_uri) { + auto account_name = Azure::Storage::_internal::ParseConnectionString( + connection_string_uri).AccountName; account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; connection_string = connection_string_uri; @@ -107,7 +110,8 @@ void AzureOptions::ConfigureServicePrincipleCredentials(const std::string& accou const std::string& client_secret) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; - service_principle_credentials_provider = std::make_shared(tenant_id, client_id, client_secret); + service_principle_credentials_provider = std::make_shared(tenant_id, client_id, client_secret); credentials_kind = AzureCredentialsKind::ServicePrincipleCredentials; } @@ -118,13 +122,14 @@ void AzureOptions::ConfigureSasCredentials(const std::string& uri) { account_blob_url = std::string(src.substr(0, first_sep)); src = internal::RemoveTrailingSlash(account_blob_url); first_sep = src.find("blob.core.windows.net"); - account_dfs_url = std::string(src.substr(0, first_sep)) + "dfs" + std::string(src.substr(first_sep + 4)) +"/"; + account_dfs_url = std::string(src.substr(0, first_sep)) + "dfs" + + std::string(src.substr(first_sep + 4)) +"/"; credentials_kind = AzureCredentialsKind::Sas; } bool AzureOptions::Equals(const AzureOptions& other) const { - return (scheme == other.scheme && - account_dfs_url == other.account_dfs_url && + return (scheme == other.scheme && + account_dfs_url == other.account_dfs_url && account_blob_url == other.account_blob_url && credentials_kind == other.credentials_kind); } @@ -153,7 +158,8 @@ AzureOptions AzureOptions::FromServicePrincipleCredential(const std::string& acc const std::string& client_id, const std::string& client_secret) { AzureOptions options; - options.ConfigureServicePrincipleCredentials(account_name, tenant_id, client_id, client_secret); + options.ConfigureServicePrincipleCredentials(account_name, tenant_id, client_id, + client_secret); return options; } @@ -286,21 +292,24 @@ std::shared_ptr GetObjectMetadata(const ObjectResult& re md->Append(std::move(k), v); } }; - for (auto prop: result) { + for (auto prop : result) { push(prop.first, prop.second); } return md; } template -Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, const std::string url) { +Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, + const std::string url) { if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(url, options.storage_credentials_provider); - } else if(options.credentials_kind == AzureCredentialsKind::ServicePrincipleCredentials) { + } else if (options.credentials_kind == + AzureCredentialsKind::ServicePrincipleCredentials) { client = std::make_shared(url, options.service_principle_credentials_provider); } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { - client = std::make_shared(T::CreateFromConnectionString(options.connection_string)); - } else if (options.credentials_kind == AzureCredentialsKind::Sas){ + client = std::make_shared(T::CreateFromConnectionString( + options.connection_string)); + } else if (options.credentials_kind == AzureCredentialsKind::Sas) { client = std::make_shared(url + options.sas_token); } else { client = std::make_shared(url); @@ -309,19 +318,24 @@ Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, } template -Status InitPathClient(std::shared_ptr& client, const AzureOptions options, const std::string path, const std::string container, const std::string path_to_file) { +Status InitPathClient(std::shared_ptr& client, const AzureOptions options, + const std::string path, const std::string container, + const std::string path_to_file) { if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(path, options.storage_credentials_provider); - } else if(options.credentials_kind == AzureCredentialsKind::ServicePrincipleCredentials) { + } else if(options.credentials_kind == + AzureCredentialsKind::ServicePrincipleCredentials) { client = std::make_shared(path, options.service_principle_credentials_provider); } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { - client = std::make_shared(T::CreateFromConnectionString(options.connection_string, container, path_to_file)); + client = std::make_shared(T::CreateFromConnectionString( + options.connection_string, container, path_to_file)); } else if (options.credentials_kind == AzureCredentialsKind::Sas) { auto src = internal::RemoveLeadingSlash(path); auto first_sep = src.find("dfs.core.windows.net"); std::string p; if (first_sep != std::string::npos) { - p = std::string(src.substr(0, first_sep)) + "blob" + std::string(src.substr(first_sep + 3)); + p = std::string(src.substr(0, first_sep)) + "blob" + + std::string(src.substr(first_sep + 3)); client = std::make_shared(p + options.sas_token); } else { client = std::make_shared(path); @@ -334,12 +348,14 @@ Status InitPathClient(std::shared_ptr& client, const AzureOptions options, co class ObjectInputFile final : public io::RandomAccessFile { public: - ObjectInputFile(std::shared_ptr pathClient, std::shared_ptr fileClient, - const io::IOContext& io_context, const AzurePath& path, int64_t size = kNoSize) - : pathClient_(std::move(pathClient)), + ObjectInputFile(std::shared_ptr + pathClient, std::shared_ptr + fileClient, const io::IOContext& io_context, const AzurePath& path, + int64_t size = kNoSize) + : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), - io_context_(io_context), - path_(path), + io_context_(io_context), + path_(path), content_length_(size) {} Status Init() { @@ -481,14 +497,15 @@ class ObjectInputFile final : public io::RandomAccessFile { class ObjectOutputStream final : public io::OutputStream { public: - ObjectOutputStream(std::shared_ptr pathClient, std::shared_ptr fileClient, - const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) + ObjectOutputStream(std::shared_ptr + pathClient, std::shared_ptr + fileClient, const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), - io_context_(io_context), + io_context_(io_context), path_(path), - metadata_(metadata){} + metadata_(metadata) {} ~ObjectOutputStream() override { // For compliance with the rest of the IO stack, Close rather than Abort, @@ -502,7 +519,7 @@ class ObjectOutputStream final : public io::OutputStream { DCHECK_GE(content_length_, 0); return Status::OK(); } - try{ + try { auto properties = pathClient_->GetProperties(); if (properties.Value.IsDirectory) { return Status::IOError("Invalid file path given"); @@ -512,7 +529,8 @@ class ObjectOutputStream final : public io::OutputStream { } catch(std::exception const& e) { //new file std::string s = ""; - fileClient_->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + fileClient_->UploadFrom(const_cast( + reinterpret_cast(&s[0])), s.size()); content_length_ = 0; } return Status::OK(); @@ -562,7 +580,8 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto result = fileClient_->UploadFrom(const_cast(reinterpret_cast(data)), nbytes).Value; + auto result = fileClient_->UploadFrom(const_cast( + reinterpret_cast(data)), nbytes).Value; pos_ += nbytes; return Status::OK(); } @@ -589,14 +608,15 @@ class ObjectOutputStream final : public io::OutputStream { class ObjectAppendStream final : public io::OutputStream { public: - ObjectAppendStream(std::shared_ptr pathClient, std::shared_ptr fileClient, - const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) + ObjectAppendStream(std::shared_ptr + pathClient, std::shared_ptr + fileClient, const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), - fileClient_(std::move(fileClient)), - io_context_(io_context), + fileClient_(std::move(fileClient)), + io_context_(io_context), path_(path), - metadata_(metadata){} + metadata_(metadata) {} ~ObjectAppendStream() override { // For compliance with the rest of the IO stack, Close rather than Abort, @@ -610,7 +630,7 @@ class ObjectAppendStream final : public io::OutputStream { DCHECK_GE(content_length_, 0); return Status::OK(); } - try{ + try { auto properties = pathClient_->GetProperties(); if (properties.Value.IsDirectory) { return Status::IOError("Invalid file path given"); @@ -667,7 +687,8 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto content = Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes); + auto content = Azure::Core::IO::MemoryBodyStream(const_cast( + reinterpret_cast(data)), nbytes); auto result = fileClient_->Append(content, 0); return Status::OK(); } @@ -697,17 +718,21 @@ TimePoint ToTimePoint(int secs) { return TimePoint(std::chrono::duration_cast(ns_count)); } -void FileObjectToInfo(const Azure::Storage::Files::DataLake::Models::PathProperties& properties, FileInfo* info) { +void FileObjectToInfo(const Azure::Storage::Files::DataLake::Models::PathProperties& + properties, FileInfo* info) { info->set_type(FileType::File); info->set_size(static_cast(properties.FileSize)); - info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(properties.LastModified))); + info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter:: + DateTimeToPosixTime(properties.LastModified))); } -void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, const Azure::DateTime dt, FileInfo* info) { +void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, + const Azure::DateTime dt, FileInfo* info) { info->set_type(type); info->set_size(size); info->set_path(path); - info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); + info->set_mtime(ToTimePoint( + Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); } } // namespace @@ -715,7 +740,8 @@ void PathInfoToFileInfo(const std::string path, const FileType type, const int64 // ----------------------------------------------------------------------- // Azure filesystem implementation -class AzureBlobFileSystem::Impl : public std::enable_shared_from_this { +class AzureBlobFileSystem::Impl : public + std::enable_shared_from_this { public: io::IOContext io_context_; std::shared_ptr gen1Client_; @@ -724,7 +750,7 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_thisGetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + isHierarchicalNamespaceEnabled = gen1Client_->GetAccountInfo() + .Value.IsHierarchicalNamespaceEnabled; return Status::OK(); } @@ -743,7 +770,7 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_thisGetFileSystemClient(container); fileSystemClient.CreateIfNotExists(); return Status::OK(); - } + } //Tests to see if a container exists Result ContainerExists(const std::string& container) { @@ -751,7 +778,7 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this pathClient_; - InitPathClient(pathClient_, options_, uri, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, uri, path.container, path.path_to_file); try { auto properties = pathClient_->GetProperties(); return properties.Value.IsDirectory; @@ -773,7 +801,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this pathClient_; - InitPathClient(pathClient_, options_, uri, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, uri, path.container, path.path_to_file); try { auto properties = pathClient_->GetProperties(); return !properties.Value.IsDirectory; @@ -782,8 +811,10 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this& path) { - auto directoryClient = gen2Client_->GetFileSystemClient(container).GetDirectoryClient(path.front()); + Status CreateEmptyDir(const std::string& container, + const std::vector& path) { + auto directoryClient = gen2Client_->GetFileSystemClient(container) + .GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { @@ -825,9 +856,11 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this 1) { - return Status::IOError("Cannot delete File, Invalid File Path, hierarchical namespace not enabled"); + return Status::IOError("Cannot delete File, Invalid File Path," + " hierarchical namespace not enabled"); } - auto blobClient = gen1Client_->GetBlobContainerClient(container).GetBlobClient(path.front()); + auto blobClient = gen1Client_->GetBlobContainerClient(container) + .GetBlobClient(path.front()); if (!FileExists(blobClient.GetUrl()).ValueOrDie()) { return Status::IOError("Cannot delete File, Invalid File Path"); } @@ -872,7 +905,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this 1 || dest_path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + if (src_path.path_to_file_parts.size() > 1 || + dest_path.path_to_file_parts.size() > 1) { + return Status::IOError("Invalid path provided, " + "hierarchical namespace not enabled"); } if (dest_path.empty() || dest_path.path_to_file_parts.empty()) { return Status::IOError("Invalid path provided at destination"); @@ -1004,14 +1041,18 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this* childrenDirs, std::vector* childrenFiles, const bool allow_not_found = false) { + Status ListPaths(const std::string& container, const std::string& path, + std::vector* childrenDirs, std::vector* + childrenFiles, const bool allow_not_found = false) { if (!isHierarchicalNamespaceEnabled) { try { auto paths = gen1Client_->GetBlobContainerClient(container).ListBlobs(); for (auto p : paths.Blobs) { - std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + std::shared_ptr + pathClient_; + InitPathClient( + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name); childrenFiles->push_back(container+"/"+p.Name); } } catch (std::exception const& e) { @@ -1025,8 +1066,11 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_thisGetFileSystemClient(container).ListPaths(false); for (auto p : paths.Paths) { - std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + std::shared_ptr + pathClient_; + InitPathClient( + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name); if (pathClient_->GetProperties().Value.IsDirectory) { childrenDirs->push_back(container+"/"+p.Name); } else { @@ -1042,14 +1086,16 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this dirs = internal::SplitAbstractPath(path); try { - Azure::Storage::Files::DataLake::DataLakeDirectoryClient dirClient = gen2Client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); - for (auto dir=dirs.begin()+1; dirGetFileSystemClient(container).GetDirectoryClient(dirs.front()); + for (auto dir = dirs.begin() + 1 ; dir < dirs.end() ; ++dir) { dirClient = dirClient.GetSubdirectoryClient(*dir); } auto paths = dirClient.ListPaths(false); for (auto p : paths.Paths) { std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + InitPathClient(pathClient_, + options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); if (pathClient_->GetProperties().Value.IsDirectory) { childrenDirs->push_back(container+"/"+p.Name); } else { @@ -1066,12 +1112,12 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this* out) { - std::vector childrenDirs; std::vector childrenFiles; - Status st = ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); - if(!st.ok()) { + Status st = ListPaths(container, path, &childrenDirs, &childrenFiles, + select.allow_not_found); + if (!st.ok()) { return st; } @@ -1080,7 +1126,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_thisGetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; GetProperties(dfs_endpoint_url+childFile, &properties); - PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, properties.LastModified, &info); + PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, + properties.LastModified, &info); out->push_back(std::move(info)); } for (const auto& childDir : childrenDirs) { @@ -1088,7 +1135,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_thisGetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; GetProperties(dfs_endpoint_url+childDir, &properties); - PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, &info); + PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, + &info); out->push_back(std::move(info)); if (select.recursive && nesting_depth < select.max_recursion) { const auto src = internal::RemoveTrailingSlash(childDir); @@ -1100,10 +1148,12 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this pathClient_; - InitPathClient(pathClient_, options_, s, path.container, path.path_to_file); + InitPathClient(pathClient_, + options_, s, path.container, path.path_to_file); if (path.path_to_file.empty()) { auto fileSystemClient = gen2Client_->GetFileSystemClient(path.container); auto props = fileSystemClient.GetProperties().Value; @@ -1121,13 +1171,13 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this& path_to_file_parts) { - + Status DeleteDirContents(const std::string& container, const std::string& path, + const std::vector& path_to_file_parts) { std::vector childrenDirs; std::vector childrenFiles; Status st = ListPaths(container, path, &childrenDirs, &childrenFiles); - if(!st.ok()) { + if (!st.ok()) { return st; } for (const auto& childFile : childrenFiles) { @@ -1145,13 +1195,14 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this> ListContainers() { auto outcome = gen2Client_->ListFileSystems(); std::vector containers; - for (auto container: outcome.FileSystems) { + for (auto container : outcome.FileSystems) { containers.push_back(container.Name); } return containers; } - Result> OpenInputFile(const std::string& s, AzureBlobFileSystem* fs) { + Result> OpenInputFile(const std::string& s, + AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty()) { @@ -1159,24 +1210,29 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this 1) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + return Status::IOError("Invalid path provided," + " hierarchical namespace not enabled"); } } if (!(FileExists(dfs_endpoint_url + path.full_path)).ValueOrDie()) { return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient(pathClient_, + options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient(fileClient_, + options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); - auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path); + auto ptr = std::make_shared(pathClient_, fileClient_, + fs->io_context(), path); RETURN_NOT_OK(ptr->Init()); return ptr; } - Result> OpenOutputStream(const std::string& s, const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + Result> OpenOutputStream(const std::string& s, + const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty() || path.path_to_file.empty()) { @@ -1185,7 +1241,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this 1) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + return Status::IOError("Invalid path provided," + " hierarchical namespace not enabled"); } endpoint_url = blob_endpoint_url; } @@ -1193,10 +1250,12 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this pathClient_; - InitPathClient(pathClient_, options_, endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient(pathClient_, + options_, endpoint_url + path.full_path, path.container, path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, options_, endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient(fileClient_, + options_, endpoint_url + path.full_path, path.container, path.path_to_file); if (path.has_parent()) { AzurePath parent_path = path.parent(); @@ -1213,27 +1272,33 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this(pathClient_, fileClient_, fs->io_context(), path, metadata); + auto ptr = std::make_shared(pathClient_, fileClient_, + fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; } - Result> OpenAppendStream(const std::string& s, const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + Result> OpenAppendStream(const std::string& s, + const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + return Status::IOError("Invalid path provided, " + "hierarchical namespace not enabled"); } } std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url+s, path.container, path.path_to_file); + InitPathClient(pathClient_, + options_, dfs_endpoint_url+s, path.container, path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, options_, dfs_endpoint_url+s, path.container, path.path_to_file); + InitPathClient(fileClient_, + options_, dfs_endpoint_url+s, path.container, path.path_to_file); - auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, metadata); + auto ptr = std::make_shared(pathClient_, fileClient_, + fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; } @@ -1251,19 +1316,23 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this 1) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + return Status::IOError("Invalid path provided, hierarchical namespace" + " not enabled"); } } if (!(FileExists(dfs_endpoint_url + info.path())).ValueOrDie()) { return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient(pathClient_, options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + InitPathClient(pathClient_, + options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + InitPathClient(fileClient_, + options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); - auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, info.size()); + auto ptr = std::make_shared(pathClient_, fileClient_, + fs->io_context(), path, info.size()); RETURN_NOT_OK(ptr->Init()); return ptr; } @@ -1272,7 +1341,8 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this(options, io_context)) { default_async_is_sync_ = false; } @@ -1293,7 +1363,8 @@ bool AzureBlobFileSystem::Equals(const FileSystem& other) const { if (other.type_name() != type_name()) { return false; } - const auto& azurefs = ::arrow::internal::checked_cast(other); + const auto& azurefs = ::arrow::internal::checked_cast + (other); return options().Equals(azurefs.options()); } @@ -1326,7 +1397,8 @@ Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { return info; } else { // It's an object - ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)); + ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(impl_->dfs_endpoint_url + + path.full_path)); if (file_exists) { // "File" object found Azure::Storage::Files::DataLake::Models::PathProperties properties; @@ -1358,7 +1430,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele // std::string url = impl_->gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties); - PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, &info); + PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, + &info); results.push_back(std::move(info)); if (select.recursive) { RETURN_NOT_OK(impl_->Walk(select, container, "", 0, &results)); @@ -1370,13 +1443,15 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele if (!impl_->isHierarchicalNamespaceEnabled) { if (base_path.path_to_file_parts.size() > 1) { if (!select.allow_not_found) { - return Status::IOError("Invalid path provided, hierarchical namespace not enabled"); + return Status::IOError("Invalid path provided, hierarchical namespace not" + " enabled"); } return results; } } - if (base_path.path_to_file.empty() && !(impl_->ContainerExists(base_path.container).ValueOrDie())) { + if (base_path.path_to_file.empty() && !(impl_->ContainerExists(base_path.container) + .ValueOrDie())) { if (!select.allow_not_found) { return Status::IOError("Invalid path provided"); } @@ -1387,7 +1462,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele return Status::IOError("Invalid path provided"); } - if (!(base_path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie())) { + if (!(base_path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + + base_path.full_path).ValueOrDie())) { if (!select.allow_not_found) { return Status::IOError("Invalid path provided"); } @@ -1395,7 +1471,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele } // Nominal case -> walk a single container - RETURN_NOT_OK(impl_->Walk(select, base_path.container, base_path.path_to_file, 0, &results)); + RETURN_NOT_OK(impl_->Walk(select, base_path.container, base_path.path_to_file, + 0, &results)); return results; } @@ -1415,7 +1492,8 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { //Hierarchical namespace not enabled type storage accounts if (!impl_->isHierarchicalNamespaceEnabled) { if (!path.path_to_file.empty()) { - return Status::IOError("Cannot create directory, storage account doesn't have hierarchical namespace enabled"); + return Status::IOError("Cannot create directory, " + "storage account doesn't have hierarchical namespace enabled"); } } if (recursive) { @@ -1468,7 +1546,8 @@ Status AzureBlobFileSystem::DeleteDir(const std::string& s) { //Hierarchical namespace not enabled type storage accounts if (!impl_->isHierarchicalNamespaceEnabled) { if (!path.path_to_file.empty()) { - return Status::IOError("Cannot delete directory, storage account doesn't have hierarchical namespace enabled"); + return Status::IOError("Cannot delete directory, storage" + "account doesn't have hierarchical namespace enabled"); } } return impl_->DeleteDir(path.container, path.path_to_file_parts); @@ -1481,7 +1560,8 @@ Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { return Status::IOError("Invalid path provided"); } - if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container).ValueOrDie())) { + if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container) + .ValueOrDie())) { return Status::IOError("Invalid path provided1"); } @@ -1489,11 +1569,13 @@ Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { return Status::IOError("Invalid path provided2"); } - if (!(path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { + if (!(path.path_to_file.empty()) && !(impl_->DirExists( + impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { return Status::IOError("Invalid path provided3"); } - return impl_->DeleteDirContents(path.container, path.path_to_file, path.path_to_file_parts); + return impl_->DeleteDirContents(path.container, + path.path_to_file, path.path_to_file_parts); } Status AzureBlobFileSystem::DeleteRootDirContents() { @@ -1542,5 +1624,5 @@ Result> AzureBlobFileSystem::OpenAppendStream( const std::string& path, const std::shared_ptr& metadata) { return impl_->OpenAppendStream(path, metadata, this); } -} -} \ No newline at end of file +} // namespace fs +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azure/azurefs.h b/cpp/src/arrow/filesystem/azure/azurefs.h index 2aa3b66fbf705..57a7fec07e147 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.h +++ b/cpp/src/arrow/filesystem/azure/azurefs.h @@ -35,7 +35,7 @@ enum class AzureCredentialsKind : int8_t { Anonymous, /// Use explicitly-provided access key pair StorageCredentials, - /// + /// Use ServicePrincipleCredentials ServicePrincipleCredentials, /// Use Sas Token to authenticate Sas, @@ -44,7 +44,7 @@ enum class AzureCredentialsKind : int8_t { }; /// Options for the AzureFileSystem implementation. -struct ARROW_EXPORT AzureOptions{ +struct ARROW_EXPORT AzureOptions { std::string scheme; std::string account_dfs_url; std::string account_blob_url; @@ -52,22 +52,24 @@ struct ARROW_EXPORT AzureOptions{ std::string sas_token; std::string connection_string; - std::shared_ptr storage_credentials_provider; - std::shared_ptr service_principle_credentials_provider; + std::shared_ptr + storage_credentials_provider; + std::shared_ptr + service_principle_credentials_provider; AzureOptions(); void ConfigureAnonymousCredentials(const std::string& account_name); - void ConfigureAccountKeyCredentials(const std::string& account_name, + void ConfigureAccountKeyCredentials(const std::string& account_name, const std::string& account_key); void ConfigureConnectionStringCredentials(const std::string& connection_string); void ConfigureServicePrincipleCredentials(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret); + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret); void ConfigureSasCredentials(const std::string& sas_token); @@ -81,16 +83,16 @@ struct ARROW_EXPORT AzureOptions{ static AzureOptions FromConnectionString(const std::string& connection_string); static AzureOptions FromServicePrincipleCredential(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret); + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret); static AzureOptions FromSas(const std::string& uri); static Result FromUri(const ::arrow::internal::Uri& uri, - std::string* out_path = NULLPTR); + std::string* out_path = NULLPTR); static Result FromUri(const std::string& uri, - std::string* out_path = NULLPTR); + std::string* out_path = NULLPTR); }; class ARROW_EXPORT AzureBlobFileSystem : public FileSystem { @@ -154,4 +156,4 @@ class ARROW_EXPORT AzureBlobFileSystem : public FileSystem { }; } // namespace fs -} // namespace arrow \ No newline at end of file +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azure/azurefs_test.cc b/cpp/src/arrow/filesystem/azure/azurefs_test.cc index dcddd5ed469b7..c987e7cc63eb7 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs_test.cc @@ -42,55 +42,56 @@ class AzureEnvTestMixin{ public: static AzureOptions options_; static std::shared_ptr fs_; - static std::shared_ptr gen2Client_; + static std::shared_ptr + gen2Client_; static std::shared_ptr gen1Client_; - AzureEnvTestMixin() {}; + AzureEnvTestMixin() {} const std::string& GetAdlsGen2AccountName() { - const static std::string accountName = [&]() -> std::string { + static const std::string accountName = [&]() -> std::string { return std::getenv("ADLS_GEN2_ACCOUNT_NAME"); }(); return accountName; } const std::string& GetAdlsGen2AccountKey() { - const static std::string accountKey = [&]() -> std::string { + static const std::string accountKey = [&]() -> std::string { return std::getenv("ADLS_GEN2_ACCOUNT_KEY"); }(); return accountKey; } const std::string& GetAdlsGen2ConnectionString() { - const static std::string connectionString = [&]() -> std::string { + static const std::string connectionString = [&]() -> std::string { return std::getenv("ADLS_GEN2_CONNECTION_STRING"); }(); return connectionString; } const std::string& GetAdlsGen2SasUrl() { - const static std::string sasUrl = [&]() -> std::string { + static const std::string sasUrl = [&]() -> std::string { return std::getenv("ADLS_GEN2_SASURL"); }(); return sasUrl; } const std::string& GetAadTenantId() { - const static std::string tenantId = [&]() -> std::string { + static const std::string tenantId = [&]() -> std::string { return std::getenv("AAD_TENANT_ID"); }(); return tenantId; } const std::string& GetAadClientId() { - const static std::string clientId = [&]() -> std::string { + static const std::string clientId = [&]() -> std::string { return std::getenv("AAD_CLIENT_ID"); }(); return clientId; } const std::string& GetAadClientSecret() { - const static std::string clientSecret = [&]() -> std::string { + static const std::string clientSecret = [&]() -> std::string { return std::getenv("AAD_CLIENT_SECRET"); }(); return clientSecret; @@ -99,7 +100,8 @@ class AzureEnvTestMixin{ // private: // const std::string& AdlsGen2AccountName = std::getenv("ADLS_GEN2_ACCOUNT_NAME"); // const std::string& AdlsGen2AccountKey = std::getenv("ADLS_GEN2_ACCOUNT_KEY"); -// const std::string& AdlsGen2ConnectionStringValue = std::getenv("ADLS_GEN2_CONNECTION_STRING"); +// const std::string& AdlsGen2ConnectionStringValue = std::getenv( +// "ADLS_GEN2_CONNECTION_STRING"); // const std::string& AdlsGen2SasUrl = std::getenv("ADLS_GEN2_SASURL"); // const std::string& AadTenantIdValue = std::getenv("AAD_TENANT_ID"); // const std::string& AadClientIdValue = std::getenv("AAD_CLIENT_ID"); @@ -108,15 +110,16 @@ class AzureEnvTestMixin{ AzureOptions AzureEnvTestMixin::options_; std::shared_ptr AzureEnvTestMixin::fs_; -std::shared_ptr AzureEnvTestMixin::gen2Client_; +std::shared_ptr + AzureEnvTestMixin::gen2Client_; std::shared_ptr AzureEnvTestMixin::gen1Client_; -class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin{ +class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin { public: - bool isHeirarchialNamespaceEnabled() { - return AzureEnvTestMixin::gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + return AzureEnvTestMixin::gen1Client_->GetAccountInfo().Value + .IsHierarchicalNamespaceEnabled; } void MakeFileSystem() { @@ -124,8 +127,12 @@ class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin const std::string& account_name = GetAdlsGen2AccountName(); AzureEnvTestMixin::options_.ConfigureAccountKeyCredentials(account_name, account_key); auto url = options_.account_dfs_url; - AzureEnvTestMixin::gen2Client_ = std::make_shared(url, options_.storage_credentials_provider); - AzureEnvTestMixin::gen1Client_ = std::make_shared(options_.account_blob_url, options_.storage_credentials_provider); + AzureEnvTestMixin::gen2Client_ = std::make_shared( + url, options_.storage_credentials_provider); + AzureEnvTestMixin::gen1Client_ = std::make_shared(options_.account_blob_url, + options_.storage_credentials_provider); auto result = AzureBlobFileSystem::Make(options_); if (!result.ok()) { ARROW_LOG(INFO) @@ -138,50 +145,62 @@ class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin void SetUp() override { { - auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container"); + auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( + "container"); fileSystemClient.CreateIfNotExists(); - fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("empty-container"); + fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( + "empty-container"); fileSystemClient.CreateIfNotExists(); } { if (isHeirarchialNamespaceEnabled()) { - auto directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container").GetDirectoryClient("emptydir"); + auto directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( + "container").GetDirectoryClient("emptydir"); directoryClient.CreateIfNotExists(); - directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container").GetDirectoryClient("somedir"); + directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container") + .GetDirectoryClient("somedir"); directoryClient.CreateIfNotExists(); directoryClient = directoryClient.GetSubdirectoryClient("subdir"); directoryClient.CreateIfNotExists(); auto fileClient = directoryClient.GetFileClient("subfile"); fileClient.CreateIfNotExists(); std::string s = "sub data"; - fileClient.UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); - fileClient = gen2Client_->GetFileSystemClient("container").GetFileClient("somefile"); + fileClient.UploadFrom(const_cast(reinterpret_cast( + &s[0])), s.size()); + fileClient = gen2Client_->GetFileSystemClient("container") + .GetFileClient("somefile"); fileClient.CreateIfNotExists(); s = "some data"; - fileClient.UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + fileClient.UploadFrom(const_cast(reinterpret_cast( + &s[0])), s.size()); } else { - auto fc = std::make_shared(options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); + auto fc = std::make_shared( + options_.account_blob_url + "container/somefile", + options_.storage_credentials_provider); std::string s = "some data"; - fc->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + fc->UploadFrom(const_cast(reinterpret_cast(&s[0])), + s.size()); } } } void TearDown() override { auto containers = AzureEnvTestMixin::gen2Client_->ListFileSystems(); - for(auto c:containers.FileSystems) { + for (auto c : containers.FileSystems) { auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient(c.Name); fileSystemClient.DeleteIfExists(); } } }; -class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin{ +class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin { public: - - void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* client, const std::string& container, - const std::string& path_to_file, const std::string& expected) { - auto pathClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, options_.storage_credentials_provider); + void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* + client, const std::string& container, const std::string& + path_to_file, const std::string& expected) { + auto pathClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, + options_.storage_credentials_provider); auto size = pathClient_->GetProperties().Value.FileSize; auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; @@ -189,11 +208,14 @@ class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin{ range.Offset = 0; range.Length = size; downloadOptions.Range = Azure::Nullable(range); - auto fileClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, options_.storage_credentials_provider); - auto result = fileClient_->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), size, downloadOptions).Value; - buf->get()->Equals(Buffer(const_cast(reinterpret_cast(&expected[0])), expected.size())); + auto fileClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, + options_.storage_credentials_provider); + auto result = fileClient_->DownloadTo(reinterpret_cast( + buf->get()->mutable_data()), size, downloadOptions).Value; + buf->get()->Equals(Buffer(const_cast(reinterpret_cast( + &expected[0])), expected.size())); } - }; TEST(TestAzureFSOptions, FromUri) { @@ -216,8 +238,10 @@ TEST(TestAzureFSOptions, FromUri) { TEST_F(TestAzureFileSystem, FromAccountKey) { AzureOptions options; - options = AzureOptions::FromAccountKey(this->GetAdlsGen2AccountKey(), this->GetAdlsGen2AccountName()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::StorageCredentials); + options = AzureOptions::FromAccountKey(this->GetAdlsGen2AccountKey(), + this->GetAdlsGen2AccountName()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind:: + StorageCredentials); ASSERT_NE(options.storage_credentials_provider, nullptr); } @@ -230,8 +254,10 @@ TEST_F(TestAzureFileSystem, FromConnectionString) { TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { AzureOptions options; - options = AzureOptions::FromServicePrincipleCredential(this->GetAdlsGen2AccountName(), this->GetAadTenantId(), this->GetAadClientId(), this->GetAadClientSecret()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ServicePrincipleCredentials); + options = AzureOptions::FromServicePrincipleCredential(this->GetAdlsGen2AccountName(), + this->GetAadTenantId(), this->GetAadClientId(), this->GetAadClientSecret()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind + ::ServicePrincipleCredentials); ASSERT_NE(options.service_principle_credentials_provider, nullptr); } @@ -271,13 +297,13 @@ TEST_F(TestAzureFileSystem, DeleteDirBlobStorage) { // Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // Nonexistent Container + // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); // C/F @@ -293,7 +319,7 @@ TEST_F(TestAzureFileSystem, DeleteDirBlobStorage) { ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); } -TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { +TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { FileSelector select; select.base_dir = "container4"; std::vector infos; @@ -301,10 +327,10 @@ TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { // Container ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteFile("")); // C/F @@ -328,7 +354,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoBlobStorage) { AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); - AssertFileInfo(fs_.get(), "", FileType::Directory); + AssertFileInfo(fs_.get(), "", FileType::Directory); auto res = fs_->OpenOutputStream("container/base.txt"); res->get()->Write("Changed the data"); @@ -342,7 +368,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoBlobStorage) { AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); } -TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { +TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { FileSelector select; std::vector infos; @@ -385,9 +411,10 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { select.allow_not_found = false; } -TEST_F(TestAzureFileSystem, MoveBlobStorage) { +TEST_F(TestAzureFileSystem, MoveBlobStorage) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); - ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", + "container/newdir/newsub")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "containerqw/ghdj")); @@ -405,8 +432,10 @@ TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", + "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", + "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); @@ -419,15 +448,23 @@ TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { res->get()->Write("Changed the data"); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/sjdj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/sjdj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container27/hshj")); } TEST_F(TestAzureFileSystem, OpenInputStreamBlobStorage) { @@ -572,13 +609,13 @@ TEST_F(TestAzureFileSystem, DeleteDirContentsBlobStorage) { // Container fs_->CreateDir("container4"); ASSERT_OK(fs_->DeleteDirContents("container4")); - AssertFileInfo(fs_.get(), "container4", FileType::Directory); + AssertFileInfo(fs_.get(), "container4", FileType::Directory); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); // C/F @@ -659,19 +696,19 @@ TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { } TEST_F(TestAzureFileSystem, DeleteDirAdlsGen2) { - FileSelector select; + FileSelector select; select.base_dir = "container4"; std::vector infos; // Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // Nonexistent Container + // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); // C/D @@ -709,10 +746,10 @@ TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { // Container ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteFile("")); // C/D @@ -749,7 +786,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); - AssertFileInfo(fs_.get(), "", FileType::Directory); + AssertFileInfo(fs_.get(), "", FileType::Directory); // "Directories" AssertFileInfo(fs_.get(), "container/emptydir", FileType::Directory); @@ -839,7 +876,8 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { TEST_F(TestAzureFileSystem, MoveAdlsGen2) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); fs_->CreateDir("container/newdir/newsub/newsubsub", true); - ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", + "container/newdir/newsub")); ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); ASSERT_OK(fs_->Move("container/emptydir", "container/emptydir1")); ASSERT_OK(fs_->Move("container/emptydir1", "container/emptydir")); @@ -861,8 +899,10 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", + "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", + "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); @@ -878,13 +918,19 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somedir/subdir/subfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/ahsj/ggws")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container27/hshj")); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile")); fs_->DeleteFile("container/somefile3"); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); @@ -1042,12 +1088,12 @@ TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { // Container ASSERT_OK(fs_->DeleteDirContents("container4")); - AssertFileInfo(fs_.get(), "container4", FileType::Directory); + AssertFileInfo(fs_.get(), "container4", FileType::Directory); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); fs_->CreateDir("container4/newdir/subdir", true); @@ -1093,4 +1139,4 @@ int main(int argc, char **argv) { ::testing::GTEST_FLAG(filter) = "*From*:*BlobStorage"; } return RUN_ALL_TESTS(); -} \ No newline at end of file +} From d3cffa2e26a0b1ccb0ad2add329bbee8106b4996 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 21 Apr 2022 10:53:15 +0530 Subject: [PATCH 03/34] ARROW-2034: [C++] Fixed formatting issues --- cpp/src/arrow/filesystem/azure/azurefs.cc | 403 +++++++++--------- cpp/src/arrow/filesystem/azure/azurefs.h | 12 +- .../arrow/filesystem/azure/azurefs_test.cc | 227 +++++----- 3 files changed, 335 insertions(+), 307 deletions(-) diff --git a/cpp/src/arrow/filesystem/azure/azurefs.cc b/cpp/src/arrow/filesystem/azure/azurefs.cc index a06f81c0287fe..a10e1642df733 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs.cc @@ -17,13 +17,13 @@ #include "arrow/filesystem/azure/azurefs.h" -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include #include #include #include @@ -65,7 +65,6 @@ #include "arrow/util/task_group.h" #include "arrow/util/thread_pool.h" - namespace arrow { using internal::Uri; @@ -86,32 +85,33 @@ void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name } void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, - const std::string& account_key) { + const std::string& account_key) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; - storage_credentials_provider = std::make_shared(account_name, account_key); + storage_credentials_provider = + std::make_shared(account_name, + account_key); credentials_kind = AzureCredentialsKind::StorageCredentials; } -void AzureOptions::ConfigureConnectionStringCredentials(const std::string& - connection_string_uri) { - auto account_name = Azure::Storage::_internal::ParseConnectionString( - connection_string_uri).AccountName; +void AzureOptions::ConfigureConnectionStringCredentials( + const std::string& connection_string_uri) { + auto account_name = + Azure::Storage::_internal::ParseConnectionString(connection_string_uri).AccountName; account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; connection_string = connection_string_uri; credentials_kind = AzureCredentialsKind::ConnectionString; } -void AzureOptions::ConfigureServicePrincipleCredentials(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret) { +void AzureOptions::ConfigureServicePrincipleCredentials( + const std::string& account_name, const std::string& tenant_id, + const std::string& client_id, const std::string& client_secret) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; - service_principle_credentials_provider = std::make_shared(tenant_id, client_id, client_secret); + service_principle_credentials_provider = + std::make_shared(tenant_id, client_id, + client_secret); credentials_kind = AzureCredentialsKind::ServicePrincipleCredentials; } @@ -123,13 +123,12 @@ void AzureOptions::ConfigureSasCredentials(const std::string& uri) { src = internal::RemoveTrailingSlash(account_blob_url); first_sep = src.find("blob.core.windows.net"); account_dfs_url = std::string(src.substr(0, first_sep)) + "dfs" + - std::string(src.substr(first_sep + 4)) +"/"; + std::string(src.substr(first_sep + 4)) + "/"; credentials_kind = AzureCredentialsKind::Sas; } bool AzureOptions::Equals(const AzureOptions& other) const { - return (scheme == other.scheme && - account_dfs_url == other.account_dfs_url && + return (scheme == other.scheme && account_dfs_url == other.account_dfs_url && account_blob_url == other.account_blob_url && credentials_kind == other.credentials_kind); } @@ -153,13 +152,12 @@ AzureOptions AzureOptions::FromConnectionString(const std::string& connection_st return options; } -AzureOptions AzureOptions::FromServicePrincipleCredential(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret) { +AzureOptions AzureOptions::FromServicePrincipleCredential( + const std::string& account_name, const std::string& tenant_id, + const std::string& client_id, const std::string& client_secret) { AzureOptions options; options.ConfigureServicePrincipleCredentials(account_name, tenant_id, client_id, - client_secret); + client_secret); return options; } @@ -176,8 +174,7 @@ Result AzureOptions::FromUri(const std::string& uri_string, return FromUri(uri, out_path); } -Result AzureOptions::FromUri(const Uri& uri, - std::string* out_path) { +Result AzureOptions::FromUri(const Uri& uri, std::string* out_path) { AzureOptions options; AZURE_ASSERT(uri.has_host()); const auto container = uri.host(); @@ -300,15 +297,15 @@ std::shared_ptr GetObjectMetadata(const ObjectResult& re template Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, - const std::string url) { + const std::string url) { if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(url, options.storage_credentials_provider); } else if (options.credentials_kind == - AzureCredentialsKind::ServicePrincipleCredentials) { + AzureCredentialsKind::ServicePrincipleCredentials) { client = std::make_shared(url, options.service_principle_credentials_provider); } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { - client = std::make_shared(T::CreateFromConnectionString( - options.connection_string)); + client = + std::make_shared(T::CreateFromConnectionString(options.connection_string)); } else if (options.credentials_kind == AzureCredentialsKind::Sas) { client = std::make_shared(url + options.sas_token); } else { @@ -319,23 +316,23 @@ Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, template Status InitPathClient(std::shared_ptr& client, const AzureOptions options, - const std::string path, const std::string container, - const std::string path_to_file) { + const std::string path, const std::string container, + const std::string path_to_file) { if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(path, options.storage_credentials_provider); - } else if(options.credentials_kind == - AzureCredentialsKind::ServicePrincipleCredentials) { + } else if (options.credentials_kind == + AzureCredentialsKind::ServicePrincipleCredentials) { client = std::make_shared(path, options.service_principle_credentials_provider); } else if (options.credentials_kind == AzureCredentialsKind::ConnectionString) { - client = std::make_shared(T::CreateFromConnectionString( - options.connection_string, container, path_to_file)); + client = std::make_shared(T::CreateFromConnectionString(options.connection_string, + container, path_to_file)); } else if (options.credentials_kind == AzureCredentialsKind::Sas) { auto src = internal::RemoveLeadingSlash(path); auto first_sep = src.find("dfs.core.windows.net"); std::string p; if (first_sep != std::string::npos) { p = std::string(src.substr(0, first_sep)) + "blob" + - std::string(src.substr(first_sep + 3)); + std::string(src.substr(first_sep + 3)); client = std::make_shared(p + options.sas_token); } else { client = std::make_shared(path); @@ -348,10 +345,10 @@ Status InitPathClient(std::shared_ptr& client, const AzureOptions options, class ObjectInputFile final : public io::RandomAccessFile { public: - ObjectInputFile(std::shared_ptr - pathClient, std::shared_ptr - fileClient, const io::IOContext& io_context, const AzurePath& path, - int64_t size = kNoSize) + ObjectInputFile( + std::shared_ptr pathClient, + std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, int64_t size = kNoSize) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), io_context_(io_context), @@ -497,10 +494,11 @@ class ObjectInputFile final : public io::RandomAccessFile { class ObjectOutputStream final : public io::OutputStream { public: - ObjectOutputStream(std::shared_ptr - pathClient, std::shared_ptr - fileClient, const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) + ObjectOutputStream( + std::shared_ptr pathClient, + std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), io_context_(io_context), @@ -526,11 +524,11 @@ class ObjectOutputStream final : public io::OutputStream { } content_length_ = properties.Value.FileSize; DCHECK_GE(content_length_, 0); - } catch(std::exception const& e) { - //new file + } catch (std::exception const& e) { + // new file std::string s = ""; - fileClient_->UploadFrom(const_cast( - reinterpret_cast(&s[0])), s.size()); + fileClient_->UploadFrom( + const_cast(reinterpret_cast(&s[0])), s.size()); content_length_ = 0; } return Status::OK(); @@ -580,8 +578,11 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto result = fileClient_->UploadFrom(const_cast( - reinterpret_cast(data)), nbytes).Value; + auto result = + fileClient_ + ->UploadFrom(const_cast(reinterpret_cast(data)), + nbytes) + .Value; pos_ += nbytes; return Status::OK(); } @@ -608,10 +609,11 @@ class ObjectOutputStream final : public io::OutputStream { class ObjectAppendStream final : public io::OutputStream { public: - ObjectAppendStream(std::shared_ptr - pathClient, std::shared_ptr - fileClient, const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) + ObjectAppendStream( + std::shared_ptr pathClient, + std::shared_ptr fileClient, + const io::IOContext& io_context, const AzurePath& path, + const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), io_context_(io_context), @@ -638,7 +640,7 @@ class ObjectAppendStream final : public io::OutputStream { content_length_ = properties.Value.FileSize; DCHECK_GE(content_length_, 0); return Status::OK(); - } catch(std::exception const& e) { + } catch (std::exception const& e) { return Status::IOError("Invalid file path given"); } } @@ -683,12 +685,12 @@ class ObjectAppendStream final : public io::OutputStream { } Status DoAppend(const void* data, int64_t nbytes, - std::shared_ptr owned_buffer = nullptr) { + std::shared_ptr owned_buffer = nullptr) { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto content = Azure::Core::IO::MemoryBodyStream(const_cast( - reinterpret_cast(data)), nbytes); + auto content = Azure::Core::IO::MemoryBodyStream( + const_cast(reinterpret_cast(data)), nbytes); auto result = fileClient_->Append(content, 0); return Status::OK(); } @@ -718,30 +720,32 @@ TimePoint ToTimePoint(int secs) { return TimePoint(std::chrono::duration_cast(ns_count)); } -void FileObjectToInfo(const Azure::Storage::Files::DataLake::Models::PathProperties& - properties, FileInfo* info) { +void FileObjectToInfo( + const Azure::Storage::Files::DataLake::Models::PathProperties& properties, + FileInfo* info) { info->set_type(FileType::File); info->set_size(static_cast(properties.FileSize)); - info->set_mtime(ToTimePoint(Azure::Core::_internal::PosixTimeConverter:: - DateTimeToPosixTime(properties.LastModified))); + info->set_mtime( + ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime( + properties.LastModified))); } void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, - const Azure::DateTime dt, FileInfo* info) { + const Azure::DateTime dt, FileInfo* info) { info->set_type(type); info->set_size(size); info->set_path(path); - info->set_mtime(ToTimePoint( - Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); + info->set_mtime( + ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); } -} // namespace +} // namespace // ----------------------------------------------------------------------- // Azure filesystem implementation -class AzureBlobFileSystem::Impl : public - std::enable_shared_from_this { +class AzureBlobFileSystem::Impl + : public std::enable_shared_from_this { public: io::IOContext io_context_; std::shared_ptr gen1Client_; @@ -758,21 +762,21 @@ class AzureBlobFileSystem::Impl : public blob_endpoint_url = options_.account_blob_url; InitServiceClient(gen1Client_, options_, blob_endpoint_url); InitServiceClient(gen2Client_, options_, dfs_endpoint_url); - isHierarchicalNamespaceEnabled = gen1Client_->GetAccountInfo() - .Value.IsHierarchicalNamespaceEnabled; + isHierarchicalNamespaceEnabled = + gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; return Status::OK(); } const AzureOptions& options() const { return options_; } - //Create a container. Successful if container already exists. + // Create a container. Successful if container already exists. Status CreateContainer(const std::string& container) { auto fileSystemClient = gen2Client_->GetFileSystemClient(container); fileSystemClient.CreateIfNotExists(); return Status::OK(); } - //Tests to see if a container exists + // Tests to see if a container exists Result ContainerExists(const std::string& container) { auto fileSystemClient = gen2Client_->GetFileSystemClient(container); try { @@ -788,7 +792,7 @@ class AzureBlobFileSystem::Impl : public ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); std::shared_ptr pathClient_; InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file); + pathClient_, options_, uri, path.container, path.path_to_file); try { auto properties = pathClient_->GetProperties(); return properties.Value.IsDirectory; @@ -802,7 +806,7 @@ class AzureBlobFileSystem::Impl : public ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); std::shared_ptr pathClient_; InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file); + pathClient_, options_, uri, path.container, path.path_to_file); try { auto properties = pathClient_->GetProperties(); return !properties.Value.IsDirectory; @@ -813,8 +817,8 @@ class AzureBlobFileSystem::Impl : public Status CreateEmptyDir(const std::string& container, const std::vector& path) { - auto directoryClient = gen2Client_->GetFileSystemClient(container) - .GetDirectoryClient(path.front()); + auto directoryClient = + gen2Client_->GetFileSystemClient(container).GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { @@ -856,11 +860,12 @@ class AzureBlobFileSystem::Impl : public } if (!isHierarchicalNamespaceEnabled) { if (path.size() > 1) { - return Status::IOError("Cannot delete File, Invalid File Path," - " hierarchical namespace not enabled"); + return Status::IOError( + "Cannot delete File, Invalid File Path," + " hierarchical namespace not enabled"); } - auto blobClient = gen1Client_->GetBlobContainerClient(container) - .GetBlobClient(path.front()); + auto blobClient = + gen1Client_->GetBlobContainerClient(container).GetBlobClient(path.front()); if (!FileExists(blobClient.GetUrl()).ValueOrDie()) { return Status::IOError("Cannot delete File, Invalid File Path"); } @@ -883,7 +888,7 @@ class AzureBlobFileSystem::Impl : public auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); - while (it != (path.end()-1)) { + while (it != (path.end() - 1)) { directoryClient = directoryClient.GetSubdirectoryClient(*it); ++it; } @@ -905,8 +910,8 @@ class AzureBlobFileSystem::Impl : public if (!isHierarchicalNamespaceEnabled) { return Status::IOError("Cannot move, Hierarchical namespace not enabled"); } - if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() - || dest_path.path_to_file.empty()) { + if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || + dest_path.path_to_file.empty()) { return Status::IOError("Invalid path provided"); } if (src_path == dest_path) { @@ -927,7 +932,7 @@ class AzureBlobFileSystem::Impl : public std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { - if ((it+1) == path.end()) { + if ((it + 1) == path.end()) { break; } directoryClient = directoryClient.GetSubdirectoryClient(*it); @@ -953,7 +958,7 @@ class AzureBlobFileSystem::Impl : public std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { - if ((it+1) == path.end()) { + if ((it + 1) == path.end()) { break; } directoryClient = directoryClient.GetSubdirectoryClient(*it); @@ -974,8 +979,8 @@ class AzureBlobFileSystem::Impl : public ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); - if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() - || dest_path.path_to_file.empty()) { + if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || + dest_path.path_to_file.empty()) { return Status::IOError("Cannot copy file, file doesn't exist at src"); } @@ -989,9 +994,10 @@ class AzureBlobFileSystem::Impl : public if (!isHierarchicalNamespaceEnabled) { if (src_path.path_to_file_parts.size() > 1 || - dest_path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided, " - "hierarchical namespace not enabled"); + dest_path.path_to_file_parts.size() > 1) { + return Status::IOError( + "Invalid path provided, " + "hierarchical namespace not enabled"); } if (dest_path.empty() || dest_path.path_to_file_parts.empty()) { return Status::IOError("Invalid path provided at destination"); @@ -1014,13 +1020,13 @@ class AzureBlobFileSystem::Impl : public if (parent_path.path_to_file.empty()) { if (!ContainerExists(parent_path.container).ValueOrDie()) { return Status::IOError("Cannot copy file '", src_path.full_path, - "': parent directory of destination does not exist"); + "': parent directory of destination does not exist"); } } else { auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); if (!(exists.ValueOrDie())) { return Status::IOError("Cannot copy file '", src_path.full_path, - "': parent directory of destination does not exist"); + "': parent directory of destination does not exist"); } } } @@ -1042,18 +1048,19 @@ class AzureBlobFileSystem::Impl : public } Status ListPaths(const std::string& container, const std::string& path, - std::vector* childrenDirs, std::vector* - childrenFiles, const bool allow_not_found = false) { + std::vector* childrenDirs, + std::vector* childrenFiles, + const bool allow_not_found = false) { if (!isHierarchicalNamespaceEnabled) { try { auto paths = gen1Client_->GetBlobContainerClient(container).ListBlobs(); for (auto p : paths.Blobs) { std::shared_ptr - pathClient_; + pathClient_; InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name); - childrenFiles->push_back(container+"/"+p.Name); + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name); + childrenFiles->push_back(container + "/" + p.Name); } } catch (std::exception const& e) { if (!allow_not_found) { @@ -1067,14 +1074,14 @@ class AzureBlobFileSystem::Impl : public auto paths = gen2Client_->GetFileSystemClient(container).ListPaths(false); for (auto p : paths.Paths) { std::shared_ptr - pathClient_; + pathClient_; InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name); + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name); if (pathClient_->GetProperties().Value.IsDirectory) { - childrenDirs->push_back(container+"/"+p.Name); + childrenDirs->push_back(container + "/" + p.Name); } else { - childrenFiles->push_back(container+"/"+p.Name); + childrenFiles->push_back(container + "/" + p.Name); } } } catch (std::exception const& e) { @@ -1086,20 +1093,21 @@ class AzureBlobFileSystem::Impl : public } std::vector dirs = internal::SplitAbstractPath(path); try { - Azure::Storage::Files::DataLake::DataLakeDirectoryClient dirClient - = gen2Client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); - for (auto dir = dirs.begin() + 1 ; dir < dirs.end() ; ++dir) { + Azure::Storage::Files::DataLake::DataLakeDirectoryClient dirClient = + gen2Client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); + for (auto dir = dirs.begin() + 1; dir < dirs.end(); ++dir) { dirClient = dirClient.GetSubdirectoryClient(*dir); } auto paths = dirClient.ListPaths(false); for (auto p : paths.Paths) { std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, dfs_endpoint_url + container + "/" + p.Name, container, p.Name); + InitPathClient( + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, + p.Name); if (pathClient_->GetProperties().Value.IsDirectory) { - childrenDirs->push_back(container+"/"+p.Name); + childrenDirs->push_back(container + "/" + p.Name); } else { - childrenFiles->push_back(container+"/"+p.Name); + childrenFiles->push_back(container + "/" + p.Name); } } } catch (std::exception const& e) { @@ -1115,8 +1123,8 @@ class AzureBlobFileSystem::Impl : public std::vector childrenDirs; std::vector childrenFiles; - Status st = ListPaths(container, path, &childrenDirs, &childrenFiles, - select.allow_not_found); + Status st = + ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); if (!st.ok()) { return st; } @@ -1125,18 +1133,18 @@ class AzureBlobFileSystem::Impl : public FileInfo info; // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - GetProperties(dfs_endpoint_url+childFile, &properties); - PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, - properties.LastModified, &info); + GetProperties(dfs_endpoint_url + childFile, &properties); + PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, + properties.LastModified, &info); out->push_back(std::move(info)); } for (const auto& childDir : childrenDirs) { FileInfo info; // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - GetProperties(dfs_endpoint_url+childDir, &properties); + GetProperties(dfs_endpoint_url + childDir, &properties); PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, - &info); + &info); out->push_back(std::move(info)); if (select.recursive && nesting_depth < select.max_recursion) { const auto src = internal::RemoveTrailingSlash(childDir); @@ -1148,12 +1156,13 @@ class AzureBlobFileSystem::Impl : public return Status::OK(); } - Status GetProperties(const std::string& s, - Azure::Storage::Files::DataLake::Models::PathProperties* properties) { + Status GetProperties( + const std::string& s, + Azure::Storage::Files::DataLake::Models::PathProperties* properties) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, s, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, s, path.container, path.path_to_file); if (path.path_to_file.empty()) { auto fileSystemClient = gen2Client_->GetFileSystemClient(path.container); auto props = fileSystemClient.GetProperties().Value; @@ -1172,7 +1181,7 @@ class AzureBlobFileSystem::Impl : public } Status DeleteDirContents(const std::string& container, const std::string& path, - const std::vector& path_to_file_parts) { + const std::vector& path_to_file_parts) { std::vector childrenDirs; std::vector childrenFiles; @@ -1191,7 +1200,6 @@ class AzureBlobFileSystem::Impl : public return Status::OK(); } - Result> ListContainers() { auto outcome = gen2Client_->ListFileSystems(); std::vector containers; @@ -1210,20 +1218,23 @@ class AzureBlobFileSystem::Impl : public } if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided," - " hierarchical namespace not enabled"); + return Status::IOError( + "Invalid path provided," + " hierarchical namespace not enabled"); } } if (!(FileExists(dfs_endpoint_url + path.full_path)).ValueOrDie()) { return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, dfs_endpoint_url + path.full_path, path.container, + path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, - options_, dfs_endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient( + fileClient_, options_, dfs_endpoint_url + path.full_path, path.container, + path.path_to_file); auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path); @@ -1231,8 +1242,9 @@ class AzureBlobFileSystem::Impl : public return ptr; } - Result> OpenOutputStream(const std::string& s, - const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + Result> OpenOutputStream( + const std::string& s, const std::shared_ptr& metadata, + AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty() || path.path_to_file.empty()) { @@ -1241,8 +1253,9 @@ class AzureBlobFileSystem::Impl : public std::string endpoint_url = dfs_endpoint_url; if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided," - " hierarchical namespace not enabled"); + return Status::IOError( + "Invalid path provided," + " hierarchical namespace not enabled"); } endpoint_url = blob_endpoint_url; } @@ -1250,55 +1263,59 @@ class AzureBlobFileSystem::Impl : public return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, - options_, endpoint_url + path.full_path, path.container, path.path_to_file); + InitPathClient( + fileClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file); if (path.has_parent()) { AzurePath parent_path = path.parent(); if (parent_path.path_to_file.empty()) { if (!ContainerExists(parent_path.container).ValueOrDie()) { return Status::IOError("Cannot write to file '", path.full_path, - "': parent directory does not exist"); + "': parent directory does not exist"); } } else { auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); if (!(exists.ValueOrDie())) { return Status::IOError("Cannot write to file '", path.full_path, - "': parent directory does not exist"); + "': parent directory does not exist"); } } } auto ptr = std::make_shared(pathClient_, fileClient_, - fs->io_context(), path, metadata); + fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; } - Result> OpenAppendStream(const std::string& s, - const std::shared_ptr& metadata, AzureBlobFileSystem* fs) { + Result> OpenAppendStream( + const std::string& s, const std::shared_ptr& metadata, + AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided, " - "hierarchical namespace not enabled"); + return Status::IOError( + "Invalid path provided, " + "hierarchical namespace not enabled"); } } std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, dfs_endpoint_url+s, path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, - options_, dfs_endpoint_url+s, path.container, path.path_to_file); + InitPathClient( + fileClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file); auto ptr = std::make_shared(pathClient_, fileClient_, - fs->io_context(), path, metadata); + fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; } @@ -1316,23 +1333,26 @@ class AzureBlobFileSystem::Impl : public if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { - return Status::IOError("Invalid path provided, hierarchical namespace" - " not enabled"); + return Status::IOError( + "Invalid path provided, hierarchical namespace" + " not enabled"); } } if (!(FileExists(dfs_endpoint_url + info.path())).ValueOrDie()) { return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient(pathClient_, - options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + InitPathClient( + pathClient_, options_, dfs_endpoint_url + info.path(), path.container, + path.path_to_file); std::shared_ptr fileClient_; - InitPathClient(fileClient_, - options_, dfs_endpoint_url + info.path(), path.container, path.path_to_file); + InitPathClient( + fileClient_, options_, dfs_endpoint_url + info.path(), path.container, + path.path_to_file); auto ptr = std::make_shared(pathClient_, fileClient_, - fs->io_context(), path, info.size()); + fs->io_context(), path, info.size()); RETURN_NOT_OK(ptr->Init()); return ptr; } @@ -1342,7 +1362,7 @@ class AzureBlobFileSystem::Impl : public }; AzureBlobFileSystem::AzureBlobFileSystem(const AzureOptions& options, - const io::IOContext& io_context) + const io::IOContext& io_context) : FileSystem(io_context), impl_(std::make_shared(options, io_context)) { default_async_is_sync_ = false; } @@ -1363,8 +1383,8 @@ bool AzureBlobFileSystem::Equals(const FileSystem& other) const { if (other.type_name() != type_name()) { return false; } - const auto& azurefs = ::arrow::internal::checked_cast - (other); + const auto& azurefs = + ::arrow::internal::checked_cast(other); return options().Equals(azurefs.options()); } @@ -1397,8 +1417,8 @@ Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { return info; } else { // It's an object - ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(impl_->dfs_endpoint_url - + path.full_path)); + ARROW_ASSIGN_OR_RAISE(bool file_exists, + impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)); if (file_exists) { // "File" object found Azure::Storage::Files::DataLake::Models::PathProperties properties; @@ -1431,7 +1451,7 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele Azure::Storage::Files::DataLake::Models::PathProperties properties; impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties); PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, - &info); + &info); results.push_back(std::move(info)); if (select.recursive) { RETURN_NOT_OK(impl_->Walk(select, container, "", 0, &results)); @@ -1443,15 +1463,16 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele if (!impl_->isHierarchicalNamespaceEnabled) { if (base_path.path_to_file_parts.size() > 1) { if (!select.allow_not_found) { - return Status::IOError("Invalid path provided, hierarchical namespace not" - " enabled"); + return Status::IOError( + "Invalid path provided, hierarchical namespace not" + " enabled"); } return results; } } - if (base_path.path_to_file.empty() && !(impl_->ContainerExists(base_path.container) - .ValueOrDie())) { + if (base_path.path_to_file.empty() && + !(impl_->ContainerExists(base_path.container).ValueOrDie())) { if (!select.allow_not_found) { return Status::IOError("Invalid path provided"); } @@ -1462,8 +1483,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele return Status::IOError("Invalid path provided"); } - if (!(base_path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url - + base_path.full_path).ValueOrDie())) { + if (!(base_path.path_to_file.empty()) && + !(impl_->DirExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie())) { if (!select.allow_not_found) { return Status::IOError("Invalid path provided"); } @@ -1471,8 +1492,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele } // Nominal case -> walk a single container - RETURN_NOT_OK(impl_->Walk(select, base_path.container, base_path.path_to_file, - 0, &results)); + RETURN_NOT_OK( + impl_->Walk(select, base_path.container, base_path.path_to_file, 0, &results)); return results; } @@ -1489,11 +1510,12 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { // Create container return impl_->CreateContainer(path.container); } - //Hierarchical namespace not enabled type storage accounts + // Hierarchical namespace not enabled type storage accounts if (!impl_->isHierarchicalNamespaceEnabled) { if (!path.path_to_file.empty()) { - return Status::IOError("Cannot create directory, " - "storage account doesn't have hierarchical namespace enabled"); + return Status::IOError( + "Cannot create directory, " + "storage account doesn't have hierarchical namespace enabled"); } } if (recursive) { @@ -1517,13 +1539,13 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { auto exists = impl_->ContainerExists(parent_path.container); if (!(exists.ValueOrDie())) { return Status::IOError("Cannot create directory '", path.full_path, - "': parent directory does not exist"); + "': parent directory does not exist"); } } else { auto exists = impl_->DirExists(impl_->dfs_endpoint_url + parent_path.full_path); if (!(exists.ValueOrDie())) { return Status::IOError("Cannot create directory '", path.full_path, - "': parent directory does not exist"); + "': parent directory does not exist"); } } } @@ -1543,11 +1565,12 @@ Status AzureBlobFileSystem::DeleteDir(const std::string& s) { return Status::IOError("Cannot delete directory, file exists at path"); } - //Hierarchical namespace not enabled type storage accounts + // Hierarchical namespace not enabled type storage accounts if (!impl_->isHierarchicalNamespaceEnabled) { if (!path.path_to_file.empty()) { - return Status::IOError("Cannot delete directory, storage" - "account doesn't have hierarchical namespace enabled"); + return Status::IOError( + "Cannot delete directory, storage" + "account doesn't have hierarchical namespace enabled"); } } return impl_->DeleteDir(path.container, path.path_to_file_parts); @@ -1560,8 +1583,8 @@ Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { return Status::IOError("Invalid path provided"); } - if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container) - .ValueOrDie())) { + if (path.path_to_file.empty() && + !(impl_->ContainerExists(path.container).ValueOrDie())) { return Status::IOError("Invalid path provided1"); } @@ -1569,13 +1592,13 @@ Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { return Status::IOError("Invalid path provided2"); } - if (!(path.path_to_file.empty()) && !(impl_->DirExists( - impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { + if (!(path.path_to_file.empty()) && + !(impl_->DirExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { return Status::IOError("Invalid path provided3"); } - return impl_->DeleteDirContents(path.container, - path.path_to_file, path.path_to_file_parts); + return impl_->DeleteDirContents(path.container, path.path_to_file, + path.path_to_file_parts); } Status AzureBlobFileSystem::DeleteRootDirContents() { @@ -1624,5 +1647,5 @@ Result> AzureBlobFileSystem::OpenAppendStream( const std::string& path, const std::shared_ptr& metadata) { return impl_->OpenAppendStream(path, metadata, this); } -} // namespace fs -} // namespace arrow +} // namespace fs +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azure/azurefs.h b/cpp/src/arrow/filesystem/azure/azurefs.h index 57a7fec07e147..3043c3db450e7 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.h +++ b/cpp/src/arrow/filesystem/azure/azurefs.h @@ -17,11 +17,11 @@ #pragma once +#include +#include #include #include #include -#include -#include #include "arrow/filesystem/filesystem.h" #include "arrow/util/macros.h" @@ -53,9 +53,9 @@ struct ARROW_EXPORT AzureOptions { std::string sas_token; std::string connection_string; std::shared_ptr - storage_credentials_provider; + storage_credentials_provider; std::shared_ptr - service_principle_credentials_provider; + service_principle_credentials_provider; AzureOptions(); @@ -155,5 +155,5 @@ class ARROW_EXPORT AzureBlobFileSystem : public FileSystem { std::shared_ptr impl_; }; -} // namespace fs -} // namespace arrow +} // namespace fs +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azure/azurefs_test.cc b/cpp/src/arrow/filesystem/azure/azurefs_test.cc index c987e7cc63eb7..a876f2450c884 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs_test.cc @@ -20,17 +20,17 @@ #include #include -#include #include +#include #include -#include "arrow/util/uri.h" #include "arrow/filesystem/test_util.h" -#include "arrow/util/key_value_metadata.h" -#include "arrow/util/logging.h" -#include "arrow/testing/gtest_util.h" #include "arrow/testing/future_util.h" +#include "arrow/testing/gtest_util.h" #include "arrow/testing/util.h" +#include "arrow/util/key_value_metadata.h" +#include "arrow/util/logging.h" +#include "arrow/util/uri.h" namespace arrow { @@ -38,12 +38,12 @@ using internal::Uri; namespace fs { -class AzureEnvTestMixin{ +class AzureEnvTestMixin { public: static AzureOptions options_; static std::shared_ptr fs_; static std::shared_ptr - gen2Client_; + gen2Client_; static std::shared_ptr gen1Client_; AzureEnvTestMixin() {} @@ -97,29 +97,28 @@ class AzureEnvTestMixin{ return clientSecret; } -// private: -// const std::string& AdlsGen2AccountName = std::getenv("ADLS_GEN2_ACCOUNT_NAME"); -// const std::string& AdlsGen2AccountKey = std::getenv("ADLS_GEN2_ACCOUNT_KEY"); -// const std::string& AdlsGen2ConnectionStringValue = std::getenv( -// "ADLS_GEN2_CONNECTION_STRING"); -// const std::string& AdlsGen2SasUrl = std::getenv("ADLS_GEN2_SASURL"); -// const std::string& AadTenantIdValue = std::getenv("AAD_TENANT_ID"); -// const std::string& AadClientIdValue = std::getenv("AAD_CLIENT_ID"); -// const std::string& AadClientSecretValue = std::getenv("AAD_CLIENT_SECRET"); + // private: + // const std::string& AdlsGen2AccountName = std::getenv("ADLS_GEN2_ACCOUNT_NAME"); + // const std::string& AdlsGen2AccountKey = std::getenv("ADLS_GEN2_ACCOUNT_KEY"); + // const std::string& AdlsGen2ConnectionStringValue = std::getenv( + // "ADLS_GEN2_CONNECTION_STRING"); + // const std::string& AdlsGen2SasUrl = std::getenv("ADLS_GEN2_SASURL"); + // const std::string& AadTenantIdValue = std::getenv("AAD_TENANT_ID"); + // const std::string& AadClientIdValue = std::getenv("AAD_CLIENT_ID"); + // const std::string& AadClientSecretValue = std::getenv("AAD_CLIENT_SECRET"); }; AzureOptions AzureEnvTestMixin::options_; std::shared_ptr AzureEnvTestMixin::fs_; std::shared_ptr - AzureEnvTestMixin::gen2Client_; + AzureEnvTestMixin::gen2Client_; std::shared_ptr AzureEnvTestMixin::gen1Client_; - class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin { public: bool isHeirarchialNamespaceEnabled() { - return AzureEnvTestMixin::gen1Client_->GetAccountInfo().Value - .IsHierarchicalNamespaceEnabled; + return AzureEnvTestMixin::gen1Client_->GetAccountInfo() + .Value.IsHierarchicalNamespaceEnabled; } void MakeFileSystem() { @@ -127,17 +126,16 @@ class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin const std::string& account_name = GetAdlsGen2AccountName(); AzureEnvTestMixin::options_.ConfigureAccountKeyCredentials(account_name, account_key); auto url = options_.account_dfs_url; - AzureEnvTestMixin::gen2Client_ = std::make_shared( - url, options_.storage_credentials_provider); - AzureEnvTestMixin::gen1Client_ = std::make_shared(options_.account_blob_url, - options_.storage_credentials_provider); + AzureEnvTestMixin::gen2Client_ = + std::make_shared( + url, options_.storage_credentials_provider); + AzureEnvTestMixin::gen1Client_ = + std::make_shared( + options_.account_blob_url, options_.storage_credentials_provider); auto result = AzureBlobFileSystem::Make(options_); if (!result.ok()) { - ARROW_LOG(INFO) - << "AzureFileSystem::Make failed, err msg is " - << result.status().ToString(); + ARROW_LOG(INFO) << "AzureFileSystem::Make failed, err msg is " + << result.status().ToString(); return; } AzureEnvTestMixin::fs_ = *result; @@ -145,41 +143,42 @@ class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin void SetUp() override { { - auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( - "container"); + auto fileSystemClient = + AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container"); fileSystemClient.CreateIfNotExists(); - fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( - "empty-container"); + fileSystemClient = + AzureEnvTestMixin::gen2Client_->GetFileSystemClient("empty-container"); fileSystemClient.CreateIfNotExists(); } { if (isHeirarchialNamespaceEnabled()) { - auto directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient( - "container").GetDirectoryClient("emptydir"); + auto directoryClient = + AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container") + .GetDirectoryClient("emptydir"); directoryClient.CreateIfNotExists(); directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container") - .GetDirectoryClient("somedir"); + .GetDirectoryClient("somedir"); directoryClient.CreateIfNotExists(); directoryClient = directoryClient.GetSubdirectoryClient("subdir"); directoryClient.CreateIfNotExists(); auto fileClient = directoryClient.GetFileClient("subfile"); fileClient.CreateIfNotExists(); std::string s = "sub data"; - fileClient.UploadFrom(const_cast(reinterpret_cast( - &s[0])), s.size()); - fileClient = gen2Client_->GetFileSystemClient("container") - .GetFileClient("somefile"); + fileClient.UploadFrom( + const_cast(reinterpret_cast(&s[0])), s.size()); + fileClient = + gen2Client_->GetFileSystemClient("container").GetFileClient("somefile"); fileClient.CreateIfNotExists(); s = "some data"; - fileClient.UploadFrom(const_cast(reinterpret_cast( - &s[0])), s.size()); + fileClient.UploadFrom( + const_cast(reinterpret_cast(&s[0])), s.size()); } else { auto fc = std::make_shared( - options_.account_blob_url + "container/somefile", - options_.storage_credentials_provider); + options_.account_blob_url + "container/somefile", + options_.storage_credentials_provider); std::string s = "some data"; fc->UploadFrom(const_cast(reinterpret_cast(&s[0])), - s.size()); + s.size()); } } } @@ -195,12 +194,14 @@ class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin { public: - void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* - client, const std::string& container, const std::string& - path_to_file, const std::string& expected) { - auto pathClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, - options_.storage_credentials_provider); + void AssertObjectContents( + Azure::Storage::Files::DataLake::DataLakeServiceClient* client, + const std::string& container, const std::string& path_to_file, + const std::string& expected) { + auto pathClient_ = + std::make_shared( + client->GetUrl() + "/" + container + "/" + path_to_file, + options_.storage_credentials_provider); auto size = pathClient_->GetProperties().Value.FileSize; auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; @@ -208,13 +209,17 @@ class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin { range.Offset = 0; range.Length = size; downloadOptions.Range = Azure::Nullable(range); - auto fileClient_ = std::make_shared(client->GetUrl() + "/"+ container + "/" + path_to_file, - options_.storage_credentials_provider); - auto result = fileClient_->DownloadTo(reinterpret_cast( - buf->get()->mutable_data()), size, downloadOptions).Value; - buf->get()->Equals(Buffer(const_cast(reinterpret_cast( - &expected[0])), expected.size())); + auto fileClient_ = + std::make_shared( + client->GetUrl() + "/" + container + "/" + path_to_file, + options_.storage_credentials_provider); + auto result = fileClient_ + ->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), + size, downloadOptions) + .Value; + buf->get()->Equals( + Buffer(const_cast(reinterpret_cast(&expected[0])), + expected.size())); } }; @@ -222,13 +227,13 @@ TEST(TestAzureFSOptions, FromUri) { AzureOptions options; Uri uri; - //Public container + // Public container ASSERT_OK(uri.Parse("https://testcontainer.dfs.core.windows.net/")); ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Anonymous); ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); - //Sas Token + // Sas Token ASSERT_OK(uri.Parse("https://testcontainer.blob.core.windows.net/?dummy_sas_token")); ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); @@ -239,9 +244,9 @@ TEST(TestAzureFSOptions, FromUri) { TEST_F(TestAzureFileSystem, FromAccountKey) { AzureOptions options; options = AzureOptions::FromAccountKey(this->GetAdlsGen2AccountKey(), - this->GetAdlsGen2AccountName()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind:: - StorageCredentials); + this->GetAdlsGen2AccountName()); + ASSERT_EQ(options.credentials_kind, + arrow::fs::AzureCredentialsKind::StorageCredentials); ASSERT_NE(options.storage_credentials_provider, nullptr); } @@ -254,10 +259,11 @@ TEST_F(TestAzureFileSystem, FromConnectionString) { TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { AzureOptions options; - options = AzureOptions::FromServicePrincipleCredential(this->GetAdlsGen2AccountName(), - this->GetAadTenantId(), this->GetAadClientId(), this->GetAadClientSecret()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind - ::ServicePrincipleCredentials); + options = AzureOptions::FromServicePrincipleCredential( + this->GetAdlsGen2AccountName(), this->GetAadTenantId(), this->GetAadClientId(), + this->GetAadClientSecret()); + ASSERT_EQ(options.credentials_kind, + arrow::fs::AzureCredentialsKind ::ServicePrincipleCredentials); ASSERT_NE(options.service_principle_credentials_provider, nullptr); } @@ -350,7 +356,7 @@ TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { } TEST_F(TestAzureFileSystem, GetFileInfoBlobStorage) { - //Containers + // Containers AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); @@ -413,8 +419,8 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { TEST_F(TestAzureFileSystem, MoveBlobStorage) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); - ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", - "container/newdir/newsub")); + ASSERT_RAISES(IOError, + fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "containerqw/ghdj")); @@ -432,10 +438,10 @@ TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", - "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", - "container/newfile")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container23/somedir/subdir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); @@ -448,23 +454,22 @@ TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { res->get()->Write("Changed the data"); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/sjdj")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/sjdj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container27/hshj")); + "container/somedir/subdir")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); } TEST_F(TestAzureFileSystem, OpenInputStreamBlobStorage) { @@ -665,7 +670,7 @@ TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { // Existing "file", should fail ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); - //C/D/D + // C/D/D AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); ASSERT_OK(fs_->CreateDir("container/somedir/subdir")); AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); @@ -673,7 +678,7 @@ TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { auto res = fs_->OpenOutputStream("container/somedir/base.txt"); res->get()->Write("Changed the data"); - //C/D/F + // C/D/F AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); ASSERT_RAISES(IOError, fs_->CreateDir("container/somedir/base.txt")); AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); @@ -706,7 +711,7 @@ TEST_F(TestAzureFileSystem, DeleteDirAdlsGen2) { // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); @@ -782,7 +787,7 @@ TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { } TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { - //Containers + // Containers AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); @@ -876,8 +881,8 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { TEST_F(TestAzureFileSystem, MoveAdlsGen2) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); fs_->CreateDir("container/newdir/newsub/newsubsub", true); - ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", - "container/newdir/newsub")); + ASSERT_RAISES(IOError, + fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); ASSERT_OK(fs_->Move("container/emptydir", "container/emptydir1")); ASSERT_OK(fs_->Move("container/emptydir1", "container/emptydir")); @@ -899,10 +904,10 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", - "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container23/somedir/subdir", - "container/newfile")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container23/somedir/subdir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); @@ -918,19 +923,19 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somedir/subdir/subfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container3435")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container27/hshj")); + "container/somedir/subdir")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile")); fs_->DeleteFile("container/somefile3"); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); @@ -1128,7 +1133,7 @@ TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { } // namespace fs } // namespace arrow -int main(int argc, char **argv) { +int main(int argc, char** argv) { auto env = new arrow::fs::SetupEnvironment(); env->MakeFileSystem(); ::testing::AddGlobalTestEnvironment(env); From af13444208bfdc6748c4ab6fe5a5366fa1604631 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Sun, 1 May 2022 22:18:20 +0530 Subject: [PATCH 04/34] Added -DARROW_AZURE in ci --- ci/scripts/cpp_build.sh | 1 + cpp/CMakeLists.txt | 7 +- cpp/cmake_modules/BuildUtils.cmake | 254 ------------------ cpp/cmake_modules/ThirdpartyToolchain.cmake | 100 ++++++- cpp/src/arrow/CMakeLists.txt | 41 --- cpp/src/arrow/filesystem/azure/CMakeLists.txt | 12 +- cpp/src/arrow/filesystem/azure/azurefs.cc | 11 - .../arrow/filesystem/azure/azurefs_test.cc | 15 +- cpp/thirdparty/versions.txt | 10 + 9 files changed, 121 insertions(+), 330 deletions(-) diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh index 2e6f35936ab89..17656a21c6e6d 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -65,6 +65,7 @@ mkdir -p ${build_dir} pushd ${build_dir} cmake \ + -DARROW_AZURE=${ARROW_AZURE:-OFF} \ -DARROW_BOOST_USE_SHARED=${ARROW_BOOST_USE_SHARED:-ON} \ -DARROW_BUILD_BENCHMARKS_REFERENCE=${ARROW_BUILD_BENCHMARKS:-OFF} \ -DARROW_BUILD_BENCHMARKS=${ARROW_BUILD_BENCHMARKS:-OFF} \ diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 7b2d941336dda..8d19d12e78ed8 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -693,9 +693,7 @@ endif() # Libraries to link statically with libarrow.so set(ARROW_LINK_LIBS) -set(ARROW_AZURE_LINK_LIBS) set(ARROW_STATIC_LINK_LIBS) -set(ARROW_AZURE_STATIC_LINK_LIBS) set(ARROW_STATIC_INSTALL_INTERFACE_LIBS) if(ARROW_USE_OPENSSL) @@ -795,8 +793,8 @@ if(ARROW_WITH_OPENTELEMETRY) endif() if (ARROW_AZURE) - list(APPEND ARROW_AZURE_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) - list(APPEND ARROW_AZURE_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) endif() if(ARROW_WITH_UTF8PROC) @@ -836,7 +834,6 @@ if(ARROW_STATIC_LINK_LIBS) endif() set(ARROW_SHARED_PRIVATE_LINK_LIBS ${ARROW_STATIC_LINK_LIBS}) -set(ARROW_AZURE_SHARED_PRIVATE_LINK_LIBS ${ARROW_AZURE_STATIC_LINK_LIBS}) # boost::filesystem is needed for S3 and Flight tests as a boost::process dependency. if(((ARROW_FLIGHT diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake index 42e45b8b02f4a..174b1c515a78b 100644 --- a/cpp/cmake_modules/BuildUtils.cmake +++ b/cpp/cmake_modules/BuildUtils.cmake @@ -46,260 +46,6 @@ if(WIN32 AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") list(APPEND ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS "BOOST_USE_WINDOWS_H=1") endif() -function(ADD_ARROW_LIB_AZURE LIB_NAME) - set(options) - set(one_value_args - BUILD_SHARED - BUILD_STATIC) - set(multi_value_args - SOURCES - STATIC_LINK_LIBS - SHARED_LINK_LIBS - DEPENDENCIES - SHARED_PRIVATE_LINK_LIBS - OUTPUT_PATH) - cmake_parse_arguments(ARG - "${options}" - "${one_value_args}" - "${multi_value_args}" - ${ARGN}) - if(ARG_UNPARSED_ARGUMENTS) - message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}") - endif() - - if(ARG_SOURCES) - set(SOURCES ${ARG_SOURCES}) - else() - set(SOURCES "${LIB_NAME}.cc") - endif() - - # Allow overriding ARROW_BUILD_SHARED and ARROW_BUILD_STATIC - if(DEFINED ARG_BUILD_SHARED) - set(BUILD_SHARED ${ARG_BUILD_SHARED}) - else() - set(BUILD_SHARED ${ARROW_BUILD_SHARED}) - endif() - if(DEFINED ARG_BUILD_STATIC) - set(BUILD_STATIC ${ARG_BUILD_STATIC}) - else() - set(BUILD_STATIC ${ARROW_BUILD_STATIC}) - endif() - if(ARG_OUTPUT_PATH) - set(OUTPUT_PATH ${ARG_OUTPUT_PATH}) - else() - set(OUTPUT_PATH ${BUILD_OUTPUT_ROOT_DIRECTORY}) - endif() - - if(WIN32 OR (CMAKE_GENERATOR STREQUAL Xcode)) - # We need to compile C++ separately for each library kind (shared and static) - # because of dllexport declarations on Windows. - # The Xcode generator doesn't reliably work with Xcode as target names are not - # guessed correctly. - set(USE_OBJLIB OFF) - else() - set(USE_OBJLIB ON) - endif() - - if(USE_OBJLIB) - # Generate a single "objlib" from all C++ modules and link - # that "objlib" into each library kind, to avoid compiling twice - add_library(${LIB_NAME}_objlib OBJECT ${SOURCES}) - # Necessary to make static linking into other shared libraries work properly - set_property(TARGET ${LIB_NAME}_objlib PROPERTY POSITION_INDEPENDENT_CODE 1) - set(LIB_DEPS $) - else() - set(LIB_DEPS ${ARG_SOURCES}) - endif() - - set(RUNTIME_INSTALL_DIR bin) - - if(BUILD_SHARED) - add_library(${LIB_NAME}_shared SHARED ${LIB_DEPS}) - - set_target_properties(${LIB_NAME}_shared - PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${OUTPUT_PATH}" - RUNTIME_OUTPUT_DIRECTORY "${OUTPUT_PATH}" - PDB_OUTPUT_DIRECTORY "${OUTPUT_PATH}" - OUTPUT_NAME ${LIB_NAME} - VERSION "${ARROW_FULL_SO_VERSION}" - SOVERSION "${ARROW_SO_VERSION}") - - target_link_libraries(${LIB_NAME}_shared LINK_PRIVATE ${ARG_SHARED_PRIVATE_LINK_LIBS}) - - install(TARGETS ${LIB_NAME}_shared - EXPORT ${LIB_NAME}_targets - RUNTIME DESTINATION ${RUNTIME_INSTALL_DIR} - LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} - ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} - INCLUDES - DESTINATION ${CMAKE_INSTALL_INCLUDEDIR}) - endif() - - if(BUILD_STATIC) - add_library(${LIB_NAME}_static SHARED ${LIB_DEPS}) - - if(MSVC_TOOLCHAIN) - set(LIB_NAME_STATIC ${LIB_NAME}_static) - else() - set(LIB_NAME_STATIC ${LIB_NAME}) - endif() - - if(ARROW_BUILD_STATIC AND WIN32) - target_compile_definitions(${LIB_NAME}_static PUBLIC ARROW_STATIC) - endif() - - set_target_properties(${LIB_NAME}_static - PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${OUTPUT_PATH}" - OUTPUT_NAME ${LIB_NAME_STATIC}) - - install(TARGETS ${LIB_NAME}_static - EXPORT ${LIB_NAME}_targets - RUNTIME DESTINATION ${RUNTIME_INSTALL_DIR} - LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR} - ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR} - INCLUDES - DESTINATION ${CMAKE_INSTALL_INCLUDEDIR}) - endif() -endfunction() - -function(ADD_TEST_CASE_AZURE REL_TEST_NAME) - set(options NO_VALGRIND ENABLED) - set(one_value_args PRECOMPILED_HEADER_LIB) - set(multi_value_args - SOURCES - PRECOMPILED_HEADERS - STATIC_LINK_LIBS - EXTRA_LINK_LIBS - EXTRA_INCLUDES - EXTRA_DEPENDENCIES - LABELS - EXTRA_LABELS - TEST_ARGUMENTS - PREFIX) - cmake_parse_arguments(ARG - "${options}" - "${one_value_args}" - "${multi_value_args}" - ${ARGN}) - if(ARG_UNPARSED_ARGUMENTS) - message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}") - endif() - - if(NO_TESTS AND NOT ARG_ENABLED) - return() - endif() - get_filename_component(TEST_NAME ${REL_TEST_NAME} NAME_WE) - - if(ARG_PREFIX) - set(TEST_NAME "${ARG_PREFIX}-${TEST_NAME}") - endif() - - if(ARG_SOURCES) - set(SOURCES ${ARG_SOURCES}) - else() - set(SOURCES "${REL_TEST_NAME}.cc") - endif() - - # Make sure the executable name contains only hyphens, not underscores - string(REPLACE "_" "-" TEST_NAME ${TEST_NAME}) - - set(TEST_PATH "${EXECUTABLE_OUTPUT_PATH}/${TEST_NAME}") - add_executable(${TEST_NAME} ${SOURCES}) - - # target_link_libraries(${TEST_NAME} PRIVATE azurefs_shared) - # With OSX and conda, we need to set the correct RPATH so that dependencies - # are found. The installed libraries with conda have an RPATH that matches - # for executables and libraries lying in $ENV{CONDA_PREFIX}/bin or - # $ENV{CONDA_PREFIX}/lib but our test libraries and executables are not - # installed there. - if(NOT "$ENV{CONDA_PREFIX}" STREQUAL "" AND APPLE) - set_target_properties(${TEST_NAME} - PROPERTIES BUILD_WITH_INSTALL_RPATH TRUE - INSTALL_RPATH_USE_LINK_PATH TRUE - INSTALL_RPATH - "${EXECUTABLE_OUTPUT_PATH};$ENV{CONDA_PREFIX}/lib") - endif() - - if(ARG_STATIC_LINK_LIBS) - # Customize link libraries - target_link_libraries(${TEST_NAME} PRIVATE ${ARG_STATIC_LINK_LIBS}) - else() - target_link_libraries(${TEST_NAME} PRIVATE ${ARROW_TEST_LINK_LIBS}) - endif() - - if(ARG_PRECOMPILED_HEADER_LIB) - reuse_precompiled_header_lib(${TEST_NAME} ${ARG_PRECOMPILED_HEADER_LIB}) - endif() - - if(ARG_PRECOMPILED_HEADERS AND ARROW_USE_PRECOMPILED_HEADERS) - target_precompile_headers(${TEST_NAME} PRIVATE ${ARG_PRECOMPILED_HEADERS}) - endif() - - if(ARG_EXTRA_LINK_LIBS) - target_link_libraries(${TEST_NAME} PRIVATE ${ARG_EXTRA_LINK_LIBS}) - endif() - - if(ARG_EXTRA_INCLUDES) - target_include_directories(${TEST_NAME} SYSTEM PUBLIC ${ARG_EXTRA_INCLUDES}) - endif() - - if(ARG_EXTRA_DEPENDENCIES) - add_dependencies(${TEST_NAME} ${ARG_EXTRA_DEPENDENCIES}) - endif() - - if(ARROW_TEST_MEMCHECK AND NOT ARG_NO_VALGRIND) - add_test(${TEST_NAME} - bash - -c - "cd '${CMAKE_SOURCE_DIR}'; \ - valgrind --suppressions=valgrind.supp --tool=memcheck --gen-suppressions=all \ - --num-callers=500 --leak-check=full --leak-check-heuristics=stdstring \ - --error-exitcode=1 ${TEST_PATH} ${ARG_TEST_ARGUMENTS}") - elseif(WIN32) - add_test(${TEST_NAME} ${TEST_PATH} ${ARG_TEST_ARGUMENTS}) - else() - add_test(${TEST_NAME} - ${BUILD_SUPPORT_DIR}/run-test.sh - ${CMAKE_BINARY_DIR} - test - ${TEST_PATH} - ${ARG_TEST_ARGUMENTS}) - endif() - - # Add test as dependency of relevant targets - add_dependencies(all-tests ${TEST_NAME}) - foreach(TARGET ${ARG_LABELS}) - add_dependencies(${TARGET} ${TEST_NAME}) - endforeach() - - set(LABELS) - list(APPEND LABELS "unittest") - if(ARG_LABELS) - list(APPEND LABELS ${ARG_LABELS}) - endif() - # EXTRA_LABELS don't create their own dependencies, they are only used - # to ease running certain test categories. - if(ARG_EXTRA_LABELS) - list(APPEND LABELS ${ARG_EXTRA_LABELS}) - endif() - - foreach(LABEL ${ARG_LABELS}) - # ensure there is a cmake target which exercises tests with this LABEL - set(LABEL_TEST_NAME "test-${LABEL}") - if(NOT TARGET ${LABEL_TEST_NAME}) - add_custom_target(${LABEL_TEST_NAME} - ctest -L "${LABEL}" --output-on-failure - USES_TERMINAL) - endif() - # ensure the test is (re)built before the LABEL test runs - add_dependencies(${LABEL_TEST_NAME} ${TEST_NAME}) - endforeach() - - set_property(TEST ${TEST_NAME} - APPEND - PROPERTY LABELS ${LABELS}) -endfunction() - function(ADD_THIRDPARTY_LIB LIB_NAME) set(options) set(one_value_args SHARED_LIB STATIC_LIB) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index 667c200b6e878..7169f10823d3f 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -423,6 +423,46 @@ else() "${THIRDPARTY_MIRROR_URL}/aws-sdk-cpp-${ARROW_AWSSDK_BUILD_VERSION}.tar.gz") endif() +if(DEFINED ENV{ARROW_AZURE_CORE_URL}) + set(AZURE_CORE_SOURCE_URL "$ENV{ARROW_AZURE_CORE_URL}") +else() + set_urls(AZURE_CORE_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-core_${ARROW_AZURE_CORE_BUILD_VERSION}.tar.gz" + ) +endif() + +if(DEFINED ENV{ARROW_AZURE_IDENTITY_URL}) + set(AZURE_IDENTITY_SOURCE_URL "$ENV{ARROW_AZURE_IDENTITY_URL}") +else() + set_urls(AZURE_IDENTITY_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-identity_${ARROW_AZURE_IDENTITY_BUILD_VERSION}.tar.gz" + ) +endif() + +if(DEFINED ENV{ARROW_AZURE_STORAGE_BLOB_URL}) + set(AZURE_STORAGE_BLOB_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_BLOB_URL}") +else() + set_urls(AZURE_STORAGE_BLOB_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-blobs_${ARROW_AZURE_STORAGE_BLOB_BUILD_VERSION}.tar.gz" + ) +endif() + +if(DEFINED ENV{ARROW_AZURE_STORAGE_COMMON_URL}) + set(AZURE_STORAGE_COMMON_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_COMMON_URL}") +else() + set_urls(AZURE_STORAGE_COMMON_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-common_${ARROW_AZURE_STORAGE_COMMON_BUILD_VERSION}.tar.gz" + ) +endif() + +if(DEFINED ENV{ARROW_AZURE_STORAGE_FILES_DATALAKE_URL}) + set(AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_FILES_DATALAKE_URL}") +else() + set_urls(AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-files-datalake_${ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_VERSION}.tar.gz" + ) +endif() + if(DEFINED ENV{ARROW_BOOST_URL}) set(BOOST_SOURCE_URL "$ENV{ARROW_BOOST_URL}") else() @@ -4542,22 +4582,74 @@ macro(build_azuresdk) string(REPLACE "-" "_" _AZURESDK_LIB_NAME_PREFIX ${_AZURESDK_LIB_UPPER}) list(APPEND AZURESDK_LIBRARIES_CPP "${_AZURESDK_LIB}-cpp") set(_AZURESDK_TARGET_NAME Azure::${_AZURESDK_LIB}) + set(_AZURESDK_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}${_AZURESDK_LIB}${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) + add_library(${_AZURESDK_TARGET_NAME} STATIC IMPORTED) + set_target_properties(${_AZURESDK_TARGET_NAME} + PROPERTIES IMPORTED_LOCATION ${_AZURESDK_STATIC_LIBRARY} + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + set("${_AZURESDK_LIB_NAME_PREFIX}_STATIC_LIBRARY" ${_AZURESDK_STATIC_LIBRARY}) list(APPEND AZURESDK_LIBRARIES ${_AZURESDK_TARGET_NAME}) endforeach() + externalproject_add(azure_core_ep + ${EP_LOG_OPTIONS} + URL ${AZURE_CORE_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_CORE_BUILD_SHA256_CHECKSUM}" + CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + BUILD_BYPRODUCTS ${AZURE_CORE_STATIC_LIBRARY}) + add_dependencies(Azure::azure-core azure_core_ep) + + externalproject_add(azure_identity_ep + ${EP_LOG_OPTIONS} + URL ${AZURE_IDENTITY_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_IDENTITY_BUILD_SHA256_CHECKSUM}" + CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + BUILD_BYPRODUCTS ${AZURE_IDENTITY_STATIC_LIBRARY}) + add_dependencies(Azure::azure-identity azure_identity_ep) + + externalproject_add(azure_storage_blobs_ep + ${EP_LOG_OPTIONS} + URL ${AZURE_STORAGE_BLOB_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_STORAGE_BLOB_BUILD_SHA256_CHECKSUM}" + CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + BUILD_BYPRODUCTS ${AZURE_STORAGE_BLOBS_STATIC_LIBRARY}) + add_dependencies(Azure::azure-storage-blobs azure_storage_blobs_ep) + + externalproject_add(azure_storage_common_ep + ${EP_LOG_OPTIONS} + URL ${AZURE_STORAGE_COMMON_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_STORAGE_COMMON_BUILD_SHA256_CHECKSUM}" + CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + BUILD_BYPRODUCTS ${AZURE_STORAGE_COMMON_STATIC_LIBRARY}) + add_dependencies(Azure::azure-storage-common azure_storage_common_ep) + + externalproject_add(azure_storage_files_datalake_ep + ${EP_LOG_OPTIONS} + URL ${AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_SHA256_CHECKSUM}" + CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + BUILD_BYPRODUCTS ${AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY}) + add_dependencies(Azure::azure-storage-files-datalake azure_storage_files_datalake_ep) + + set_property(TARGET Azure::azure-core + APPEND + PROPERTY INTERFACE_LINK_LIBRARIES CURL::libcurl LibXml2::LibXml2) + set(AZURESDK_LINK_LIBRARIES ${AZURESDK_LIBRARIES}) endmacro() if(ARROW_AZURE) + # TODO - use resolve_dependency build_azuresdk() - foreach(AZURESDK_LIBRARY_CPP ${AZURESDK_LIBRARIES_CPP}) find_package(${AZURESDK_LIBRARY_CPP} CONFIG REQUIRED) endforeach() - include_directories(SYSTEM ${AZURESDK_INCLUDE_DIR}) - message(STATUS "Found AZURE SDK headers: ${AZURESDK_INCLUDE_DIR}") - message(STATUS "Found AZURE SDK libraries: ${AZURESDK_LINK_LIBRARIES}") + message(STATUS "Found Azure SDK headers: ${AZURESDK_INCLUDE_DIR}") + message(STATUS "Found Azure SDK libraries: ${AZURESDK_LINK_LIBRARIES}") endif() message(STATUS "All bundled static libraries: ${ARROW_BUNDLED_STATIC_LIBS}") diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 9f26b7b4747b6..daea61e531b42 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -68,47 +68,6 @@ function(ADD_ARROW_TEST REL_TEST_NAME) ${ARG_UNPARSED_ARGUMENTS}) endfunction() -function(ADD_ARROW_TEST_AZURE REL_TEST_NAME) - set(options) - set(one_value_args PREFIX) - set(multi_value_args LABELS PRECOMPILED_HEADERS) - cmake_parse_arguments(ARG - "${options}" - "${one_value_args}" - "${multi_value_args}" - ${ARGN}) - - if(ARG_PREFIX) - set(PREFIX ${ARG_PREFIX}) - else() - set(PREFIX "arrow") - endif() - - if(ARG_LABELS) - set(LABELS ${ARG_LABELS}) - else() - set(LABELS "arrow-tests") - endif() - - # Because of https://gitlab.kitware.com/cmake/cmake/issues/20289, - # we must generate the precompiled header on an executable target. - # Do that on the first unit test target (here "arrow-array-test") - # and reuse the PCH for the other tests. - if(ARG_PRECOMPILED_HEADERS) - set(PCH_ARGS PRECOMPILED_HEADERS ${ARG_PRECOMPILED_HEADERS}) - else() - set(PCH_ARGS PRECOMPILED_HEADER_LIB "arrow-array-test") - endif() - - add_test_case_azure(${REL_TEST_NAME} - PREFIX - ${PREFIX} - LABELS - ${LABELS} - ${PCH_ARGS} - ${ARG_UNPARSED_ARGUMENTS}) -endfunction() - function(ADD_ARROW_FUZZ_TARGET REL_FUZZING_NAME) set(options) set(one_value_args PREFIX) diff --git a/cpp/src/arrow/filesystem/azure/CMakeLists.txt b/cpp/src/arrow/filesystem/azure/CMakeLists.txt index a3c0a43e3cd65..985f37cd081d3 100644 --- a/cpp/src/arrow/filesystem/azure/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/azure/CMakeLists.txt @@ -18,19 +18,21 @@ set(CMAKE_CXX_STANDARD 14) set(CMAKE_CXX_STANDARD_REQUIRED ON) -add_arrow_lib_azure(azurefs +add_arrow_lib(azurefs + SOURCES + azurefs.cc SHARED_LINK_LIBS - ${ARROW_AZURE_LINK_LIBS} + ${ARROW_LINK_LIBS} SHARED_PRIVATE_LINK_LIBS - ${ARROW_AZURE_SHARED_PRIVATE_LINK_LIBS} + ${ARROW_SHARED_PRIVATE_LINK_LIBS} STATIC_LINK_LIBS - ${ARROW_AZURE_STATIC_LINK_LIBS}) + ${ARROW_STATIC_LINK_LIBS}) arrow_install_all_headers("arrow/filesystem/azure") set(TEST_LIBS_AZURE ${ARROW_TEST_LINK_LIBS}) list(APPEND TEST_LIBS_AZURE azurefs_shared) -add_arrow_test_azure(azurefs_test EXTRA_LABELS filesystem +add_arrow_test(azurefs_test EXTRA_LABELS filesystem STATIC_LINK_LIBS ${TEST_LIBS_AZURE} ) \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/azure/azurefs.cc b/cpp/src/arrow/filesystem/azure/azurefs.cc index a10e1642df733..cc390b6952677 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs.cc @@ -34,16 +34,6 @@ #include #include -#ifdef _WIN32 -// Undefine preprocessor macros that interfere with AWS function / method names -#ifdef GetMessage -#undef GetMessage -#endif -#ifdef GetObject -#undef GetObject -#endif -#endif - #include "arrow/util/windows_fixup.h" #include "arrow/buffer.h" @@ -176,7 +166,6 @@ Result AzureOptions::FromUri(const std::string& uri_string, Result AzureOptions::FromUri(const Uri& uri, std::string* out_path) { AzureOptions options; - AZURE_ASSERT(uri.has_host()); const auto container = uri.host(); auto path = uri.path(); if (container.empty()) { diff --git a/cpp/src/arrow/filesystem/azure/azurefs_test.cc b/cpp/src/arrow/filesystem/azure/azurefs_test.cc index a876f2450c884..539b415498080 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azure/azurefs_test.cc @@ -48,6 +48,8 @@ class AzureEnvTestMixin { AzureEnvTestMixin() {} + // Test account info is fetched from environment variables. + // To run azure tests, first set these environment variables. const std::string& GetAdlsGen2AccountName() { static const std::string accountName = [&]() -> std::string { return std::getenv("ADLS_GEN2_ACCOUNT_NAME"); @@ -96,16 +98,6 @@ class AzureEnvTestMixin { }(); return clientSecret; } - - // private: - // const std::string& AdlsGen2AccountName = std::getenv("ADLS_GEN2_ACCOUNT_NAME"); - // const std::string& AdlsGen2AccountKey = std::getenv("ADLS_GEN2_ACCOUNT_KEY"); - // const std::string& AdlsGen2ConnectionStringValue = std::getenv( - // "ADLS_GEN2_CONNECTION_STRING"); - // const std::string& AdlsGen2SasUrl = std::getenv("ADLS_GEN2_SASURL"); - // const std::string& AadTenantIdValue = std::getenv("AAD_TENANT_ID"); - // const std::string& AadClientIdValue = std::getenv("AAD_CLIENT_ID"); - // const std::string& AadClientSecretValue = std::getenv("AAD_CLIENT_SECRET"); }; AzureOptions AzureEnvTestMixin::options_; @@ -114,6 +106,9 @@ std::shared_ptr AzureEnvTestMixin::gen2Client_; std::shared_ptr AzureEnvTestMixin::gen1Client_; +// A global test environment for setting up directories and files inside the specified +// container. + class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin { public: bool isHeirarchialNamespaceEnabled() { diff --git a/cpp/thirdparty/versions.txt b/cpp/thirdparty/versions.txt index 7acecd1a5307f..26f1d63b3ee0d 100644 --- a/cpp/thirdparty/versions.txt +++ b/cpp/thirdparty/versions.txt @@ -33,6 +33,16 @@ ARROW_AWS_C_COMMON_BUILD_VERSION=v0.6.9 ARROW_AWS_C_COMMON_BUILD_SHA256_CHECKSUM=928a3e36f24d1ee46f9eec360ec5cebfe8b9b8994fe39d4fa74ff51aebb12717 ARROW_AWS_C_EVENT_STREAM_BUILD_VERSION=v0.1.5 ARROW_AWS_C_EVENT_STREAM_BUILD_SHA256_CHECKSUM=f1b423a487b5d6dca118bfc0d0c6cc596dc476b282258a3228e73a8f730422d4 +ARROW_AZURE_CORE_BUILD_VERSION=1.5.0 +ARROW_AZURE_CORE_BUILD_SHA256_CHECKSUM=dab2caa54d062b61dbe982e29a4f1fcc70216b51b038a807763712a40dd258e9 +ARROW_AZURE_IDENTITY_BUILD_VERSION=1.2.0 +ARROW_AZURE_IDENTITY_BUILD_SHA256_CHECKSUM=ad4702890c25f956c59a63be4571a08ae0690fa6d2bfbebf326d0fd2e9b72945 +ARROW_AZURE_STORAGE_BLOB_BUILD_VERSION=12.4.0 +ARROW_AZURE_STORAGE_BLOB_BUILD_SHA256_CHECKSUM=ce77055ff5e1b88826a89a29399ffbdcdc77beca1eae61c81f34a3f6e0a20715 +ARROW_AZURE_STORAGE_COMMON_BUILD_VERSION=12.2.3 +ARROW_AZURE_STORAGE_COMMON_BUILD_SHA256_CHECKSUM=2d58e9c314b1b32f7d09880239a4ecce6686ed6df236a58f681ae5d526ed6201 +ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_VERSION=12.3.1 +ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_SHA256_CHECKSUM=a5b74076a751d7cfaf7c56674a40ce2792c4fab9add18758fab1fe091d00baff ARROW_BOOST_BUILD_VERSION=1.75.0 ARROW_BOOST_BUILD_SHA256_CHECKSUM=267e04a7c0bfe85daf796dedc789c3a27a76707e1c968f0a2a87bb96331e2b61 ARROW_BROTLI_BUILD_VERSION=v1.0.9 From 1026e157e70e2efb5fe23d3c64812073f6502bc1 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 9 May 2022 15:55:07 +0530 Subject: [PATCH 05/34] Added CXX_STANDARD and CXX_STANDARD_REQUIRED --- cpp/src/arrow/CMakeLists.txt | 4 -- cpp/src/arrow/filesystem/CMakeLists.txt | 22 ++++++++ cpp/src/arrow/filesystem/azure/CMakeLists.txt | 38 -------------- .../arrow/filesystem/{azure => }/azurefs.cc | 52 ++++++++++++++++--- .../arrow/filesystem/{azure => }/azurefs.h | 2 + .../filesystem/{azure => }/azurefs_test.cc | 0 6 files changed, 69 insertions(+), 49 deletions(-) delete mode 100644 cpp/src/arrow/filesystem/azure/CMakeLists.txt rename cpp/src/arrow/filesystem/{azure => }/azurefs.cc (97%) rename cpp/src/arrow/filesystem/{azure => }/azurefs.h (98%) rename cpp/src/arrow/filesystem/{azure => }/azurefs_test.cc (100%) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index daea61e531b42..2933457287407 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -485,10 +485,6 @@ if(ARROW_FILESYSTEM) SKIP_UNITY_BUILD_INCLUSION ON) endif() - if(ARROW_AZURE) - add_subdirectory(filesystem/azure) - endif() - list(APPEND ARROW_TESTING_SRCS filesystem/test_util.cc) endif() diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index c301a57bade68..dd461a02ce64f 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -32,6 +32,28 @@ if(ARROW_GCS) add_arrow_test(gcsfs_test EXTRA_LABELS filesystem) endif() +if(ARROW_AZURE) + add_arrow_lib(azurefs + SOURCES + azurefs.cc + SHARED_LINK_LIBS + ${ARROW_LINK_LIBS} + SHARED_PRIVATE_LINK_LIBS + ${ARROW_SHARED_PRIVATE_LINK_LIBS} + STATIC_LINK_LIBS + ${ARROW_STATIC_LINK_LIBS}) + + set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) + + set(TEST_LIBS_AZURE ${ARROW_TEST_LINK_LIBS}) + list(APPEND TEST_LIBS_AZURE azurefs_shared) + add_arrow_test(azurefs_test EXTRA_LABELS filesystem + STATIC_LINK_LIBS + ${TEST_LIBS_AZURE} + ) + set_target_properties(arrow-azurefs-test PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) +endif() + if(ARROW_S3) add_arrow_test(s3fs_test SOURCES diff --git a/cpp/src/arrow/filesystem/azure/CMakeLists.txt b/cpp/src/arrow/filesystem/azure/CMakeLists.txt deleted file mode 100644 index 985f37cd081d3..0000000000000 --- a/cpp/src/arrow/filesystem/azure/CMakeLists.txt +++ /dev/null @@ -1,38 +0,0 @@ -# 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. - -set(CMAKE_CXX_STANDARD 14) -set(CMAKE_CXX_STANDARD_REQUIRED ON) - -add_arrow_lib(azurefs - SOURCES - azurefs.cc - SHARED_LINK_LIBS - ${ARROW_LINK_LIBS} - SHARED_PRIVATE_LINK_LIBS - ${ARROW_SHARED_PRIVATE_LINK_LIBS} - STATIC_LINK_LIBS - ${ARROW_STATIC_LINK_LIBS}) - -arrow_install_all_headers("arrow/filesystem/azure") - -set(TEST_LIBS_AZURE ${ARROW_TEST_LINK_LIBS}) -list(APPEND TEST_LIBS_AZURE azurefs_shared) -add_arrow_test(azurefs_test EXTRA_LABELS filesystem - STATIC_LINK_LIBS - ${TEST_LIBS_AZURE} - ) \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/azure/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc similarity index 97% rename from cpp/src/arrow/filesystem/azure/azurefs.cc rename to cpp/src/arrow/filesystem/azurefs.cc index cc390b6952677..07773a4194dfa 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -68,6 +68,44 @@ static const char kSep = '/'; AzureOptions::AzureOptions() {} +std::string GetAccountNameFromConnectionString(const std::string& connectionString) { + std::map connectionStringMap; + std::string::const_iterator cur = connectionString.begin(); + + while (cur != connectionString.end()) { + auto key_begin = cur; + auto key_end = std::find(cur, connectionString.end(), '='); + std::string key = std::string(key_begin, key_end); + cur = key_end; + if (cur != connectionString.end()) { + ++cur; + } + auto value_begin = cur; + auto value_end = std::find(cur, connectionString.end(), ';'); + std::string value = std::string(value_begin, value_end); + cur = value_end; + if (cur != connectionString.end()) { + ++cur; + } + if (!key.empty() || !value.empty()) { + connectionStringMap[std::move(key)] = std::move(value); + } + } + + auto getWithDefault = [](const std::map& m, + const std::string& key, + const std::string& defaultValue = std::string()) { + auto ite = m.find(key); + return ite == m.end() ? defaultValue : ite->second; + }; + + std::string accountName = getWithDefault(connectionStringMap, "AccountName"); + if (accountName.empty()) { + throw std::runtime_error("Cannot find account name in connection string."); + } + return accountName; +} + void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; @@ -86,8 +124,7 @@ void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_nam void AzureOptions::ConfigureConnectionStringCredentials( const std::string& connection_string_uri) { - auto account_name = - Azure::Storage::_internal::ParseConnectionString(connection_string_uri).AccountName; + auto account_name = GetAccountNameFromConnectionString(connection_string_uri); account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; connection_string = connection_string_uri; @@ -714,9 +751,9 @@ void FileObjectToInfo( FileInfo* info) { info->set_type(FileType::File); info->set_size(static_cast(properties.FileSize)); - info->set_mtime( - ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime( - properties.LastModified))); + info->set_mtime(ToTimePoint(std::chrono::duration_cast( + properties.LastModified - Azure::DateTime(1970)) + .count())); } void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, @@ -724,8 +761,9 @@ void PathInfoToFileInfo(const std::string path, const FileType type, const int64 info->set_type(type); info->set_size(size); info->set_path(path); - info->set_mtime( - ToTimePoint(Azure::Core::_internal::PosixTimeConverter::DateTimeToPosixTime(dt))); + info->set_mtime(ToTimePoint( + std::chrono::duration_cast(dt - Azure::DateTime(1970)) + .count())); } } // namespace diff --git a/cpp/src/arrow/filesystem/azure/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h similarity index 98% rename from cpp/src/arrow/filesystem/azure/azurefs.h rename to cpp/src/arrow/filesystem/azurefs.h index 3043c3db450e7..9c4d8bbeb7f64 100644 --- a/cpp/src/arrow/filesystem/azure/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -59,6 +59,8 @@ struct ARROW_EXPORT AzureOptions { AzureOptions(); + std::string GetAccountNameFromConnectionString(const std::string& connectionString); + void ConfigureAnonymousCredentials(const std::string& account_name); void ConfigureAccountKeyCredentials(const std::string& account_name, diff --git a/cpp/src/arrow/filesystem/azure/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc similarity index 100% rename from cpp/src/arrow/filesystem/azure/azurefs_test.cc rename to cpp/src/arrow/filesystem/azurefs_test.cc From 5f8b82ab4de84f251aa10384d8b89c232592bc8c Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 26 May 2022 12:57:18 +0530 Subject: [PATCH 06/34] Added mocked test file --- ci/appveyor-cpp-build.bat | 1 + cpp/src/arrow/filesystem/CMakeLists.txt | 8 +- cpp/src/arrow/filesystem/azurefs.cc | 5 +- cpp/src/arrow/filesystem/azurefs_mock.cc | 1163 ++++++++++++++++++++++ cpp/src/arrow/filesystem/azurefs_mock.h | 95 ++ cpp/src/arrow/filesystem/azurefs_test.cc | 748 ++------------ 6 files changed, 1345 insertions(+), 675 deletions(-) create mode 100644 cpp/src/arrow/filesystem/azurefs_mock.cc create mode 100644 cpp/src/arrow/filesystem/azurefs_mock.h diff --git a/ci/appveyor-cpp-build.bat b/ci/appveyor-cpp-build.bat index a69e7a665bd5f..cc8413f16a222 100644 --- a/ci/appveyor-cpp-build.bat +++ b/ci/appveyor-cpp-build.bat @@ -106,6 +106,7 @@ cmake -G "%GENERATOR%" %CMAKE_ARGS% ^ -DARROW_PARQUET=ON ^ -DARROW_PYTHON=ON ^ -DARROW_S3=%ARROW_S3% ^ + -DARROW_AZURE=ON ^ -DARROW_VERBOSE_THIRDPARTY_BUILD=OFF ^ -DARROW_WITH_BROTLI=ON ^ -DARROW_WITH_LZ4=ON ^ diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index dd461a02ce64f..3fdc1e3ab6279 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -33,9 +33,15 @@ if(ARROW_GCS) endif() if(ARROW_AZURE) + set(AZURE_SRCS) + list(APPEND + AZURE_SRCS + azurefs_mock.cc + azurefs.cc) + add_arrow_lib(azurefs SOURCES - azurefs.cc + ${AZURE_SRCS} SHARED_LINK_LIBS ${ARROW_LINK_LIBS} SHARED_PRIVATE_LINK_LIBS diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 07773a4194dfa..bf348a2e7942d 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/filesystem/azure/azurefs.h" +#include "arrow/filesystem/azurefs.h" #include #include @@ -68,7 +68,8 @@ static const char kSep = '/'; AzureOptions::AzureOptions() {} -std::string GetAccountNameFromConnectionString(const std::string& connectionString) { +std::string AzureOptions::GetAccountNameFromConnectionString( + const std::string& connectionString) { std::map connectionStringMap; std::string::const_iterator cur = connectionString.begin(); diff --git a/cpp/src/arrow/filesystem/azurefs_mock.cc b/cpp/src/arrow/filesystem/azurefs_mock.cc new file mode 100644 index 0000000000000..945b8e8859a0c --- /dev/null +++ b/cpp/src/arrow/filesystem/azurefs_mock.cc @@ -0,0 +1,1163 @@ +// 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_mock.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "arrow/buffer.h" +#include "arrow/buffer_builder.h" +#include "arrow/filesystem/mockfs.h" +#include "arrow/filesystem/path_util.h" +#include "arrow/filesystem/util_internal.h" +#include "arrow/io/interfaces.h" +#include "arrow/io/memory.h" +#include "arrow/util/async_generator.h" +#include "arrow/util/future.h" +#include "arrow/util/logging.h" +#include "arrow/util/string_view.h" +#include "arrow/util/variant.h" +#include "arrow/util/windows_fixup.h" + +namespace arrow { +namespace fs { +namespace internal { + +namespace { + +Status ValidatePath(util::string_view s) { + if (internal::IsLikelyUri(s)) { + return Status::Invalid("Expected a filesystem path, got a URI: '", s, "'"); + } + return Status::OK(); +} + +//////////////////////////////////////////////////////////////////////////// +// Filesystem structure + +class Entry; + +struct AzurePath { + std::string full_path; + std::string container; + std::string path_to_file; + std::vector path_to_file_parts; + + static Result FromString(const std::string& s) { + // https://synapsemladlsgen2.dfs.core.windows.net/synapsemlfs/testdir/testfile.txt + // container = synapsemlfs + // account_name = synapsemladlsgen2 + // path_to_file = testdir/testfile.txt + // path_to_file_parts = [testdir, testfile.txt] + + // Expected input here => s = /synapsemlfs/testdir/testfile.txt + auto src = internal::RemoveTrailingSlash(s); + if (src.starts_with("https:") || src.starts_with("http::")) { + RemoveSchemeFromUri(src); + } + auto first_sep = src.find_first_of(kSep); + if (first_sep == 0) { + return Status::Invalid("Path cannot start with a separator ('", s, "')"); + } + if (first_sep == std::string::npos) { + return AzurePath{std::string(src), std::string(src), "", {}}; + } + AzurePath path; + path.full_path = std::string(src); + path.container = std::string(src.substr(0, first_sep)); + path.path_to_file = std::string(src.substr(first_sep + 1)); + path.path_to_file_parts = internal::SplitAbstractPath(path.path_to_file); + RETURN_NOT_OK(Validate(&path)); + return path; + } + + static void RemoveSchemeFromUri(nonstd::sv_lite::string_view& s) { + auto first = s.find(".core.windows.net"); + s = s.substr(first + 18, s.length()); + } + + static Status Validate(const AzurePath* path) { + auto result = internal::ValidateAbstractPathParts(path->path_to_file_parts); + if (!result.ok()) { + return Status::Invalid(result.message(), " in path ", path->full_path); + } else { + return result; + } + } + + AzurePath parent() const { + DCHECK(!path_to_file_parts.empty()); + auto parent = AzurePath{"", container, "", path_to_file_parts}; + parent.path_to_file_parts.pop_back(); + parent.path_to_file = internal::JoinAbstractPath(parent.path_to_file_parts); + if (parent.path_to_file.empty()) { + parent.full_path = parent.container; + } else { + parent.full_path = parent.container + kSep + parent.path_to_file; + } + return parent; + } + + bool has_parent() const { return !path_to_file.empty(); } + + bool empty() const { return container.empty() && path_to_file.empty(); } + + bool operator==(const AzurePath& other) const { + return container == other.container && path_to_file == other.path_to_file; + } +}; + +struct File { + TimePoint mtime; + std::string name; + std::string path; + std::shared_ptr data; + std::shared_ptr metadata; + + File(TimePoint mtime, std::string name, std::string path) + : mtime(mtime), name(std::move(name)), path(std::move(path)) {} + + int64_t size() const { return data ? data->size() : 0; } + + explicit operator util::string_view() const { + if (data) { + return util::string_view(*data); + } else { + return ""; + } + } +}; + +struct Directory { + std::string name; + std::string path; + TimePoint mtime; + std::map> entries; + + Directory(std::string name, std::string path, TimePoint mtime) + : name(std::move(name)), path(std::move(path)), mtime(mtime) {} + Directory(Directory&& other) noexcept + : name(std::move(other.name)), + path(other.path), + mtime(other.mtime), + entries(std::move(other.entries)) {} + + Directory& operator=(Directory&& other) noexcept { + name = std::move(other.name); + mtime = other.mtime; + path = std::move(other.path); + entries = std::move(other.entries); + return *this; + } + + Entry* Find(const std::string& s) { + auto it = entries.find(s); + if (it != entries.end()) { + return it->second.get(); + } else { + return nullptr; + } + } + + bool CreateEntry(const std::string& s, std::unique_ptr entry) { + DCHECK(!s.empty()); + auto p = entries.emplace(s, std::move(entry)); + return p.second; + } + + void AssignEntry(const std::string& s, std::unique_ptr entry) { + DCHECK(!s.empty()); + entries[s] = std::move(entry); + } + + bool DeleteEntry(const std::string& s) { return entries.erase(s) > 0; } + + private: + ARROW_DISALLOW_COPY_AND_ASSIGN(Directory); +}; + +// A filesystem entry +using EntryBase = util::Variant; + +class Entry : public EntryBase { + public: + Entry(Entry&&) = default; + Entry& operator=(Entry&&) = default; + explicit Entry(Directory&& v) : EntryBase(std::move(v)) {} + explicit Entry(File&& v) : EntryBase(std::move(v)) {} + + bool is_dir() const { return util::holds_alternative(*this); } + + bool is_file() const { return util::holds_alternative(*this); } + + Directory& as_dir() { return util::get(*this); } + + File& as_file() { return util::get(*this); } + + // Get info for this entry. Note the path() property isn't set. + FileInfo GetInfo() { + FileInfo info; + if (is_dir()) { + Directory& dir = as_dir(); + info.set_type(FileType::Directory); + info.set_mtime(dir.mtime); + info.set_path(dir.path); + } else { + DCHECK(is_file()); + File& file = as_file(); + info.set_type(FileType::File); + info.set_mtime(file.mtime); + info.set_size(file.size()); + info.set_path(file.path); + } + return info; + } + + // Get info for this entry, knowing the parent path. + FileInfo GetInfo(const std::string& base_path) { + FileInfo info; + if (is_dir()) { + Directory& dir = as_dir(); + info.set_type(FileType::Directory); + info.set_mtime(dir.mtime); + info.set_path(ConcatAbstractPath(base_path, dir.name)); + } else { + DCHECK(is_file()); + File& file = as_file(); + info.set_type(FileType::File); + info.set_mtime(file.mtime); + info.set_size(file.size()); + info.set_path(ConcatAbstractPath(base_path, file.name)); + } + return info; + } + + // Set the entry name + void SetName(const std::string& name) { + if (is_dir()) { + as_dir().name = name; + } else { + DCHECK(is_file()); + as_file().name = name; + } + } + + private: + ARROW_DISALLOW_COPY_AND_ASSIGN(Entry); +}; + +//////////////////////////////////////////////////////////////////////////// +// Streams + +class MockFSOutputStream : public io::OutputStream { + public: + MockFSOutputStream(File* file, MemoryPool* pool) + : file_(file), builder_(pool), closed_(false) {} + + ~MockFSOutputStream() override = default; + + // Implement the OutputStream interface + Status Close() override { + if (!closed_) { + RETURN_NOT_OK(builder_.Finish(&file_->data)); + closed_ = true; + } + return Status::OK(); + } + + Status Abort() override { + if (!closed_) { + // MockFSOutputStream is mainly used for debugging and testing, so + // mark an aborted file's contents explicitly. + std::stringstream ss; + ss << "MockFSOutputStream aborted after " << file_->size() << " bytes written"; + file_->data = Buffer::FromString(ss.str()); + closed_ = true; + } + return Status::OK(); + } + + bool closed() const override { return closed_; } + + Result Tell() const override { + if (closed_) { + return Status::Invalid("Invalid operation on closed stream"); + } + return builder_.length(); + } + + Status Write(const void* data, int64_t nbytes) override { + if (closed_) { + return Status::Invalid("Invalid operation on closed stream"); + } + return builder_.Append(data, nbytes); + } + + protected: + File* file_; + BufferBuilder builder_; + bool closed_; +}; + +class MockFSInputStream : public io::BufferReader { + public: + explicit MockFSInputStream(const File& file) + : io::BufferReader(file.data), metadata_(file.metadata) {} + + Result> ReadMetadata() override { + return metadata_; + } + + protected: + std::shared_ptr metadata_; +}; + +} // namespace + +std::ostream& operator<<(std::ostream& os, const MockDirInfo& di) { + return os << "'" << di.full_path << "' [mtime=" << di.mtime.time_since_epoch().count() + << "]"; +} + +std::ostream& operator<<(std::ostream& os, const MockFileInfo& di) { + return os << "'" << di.full_path << "' [mtime=" << di.mtime.time_since_epoch().count() + << ", size=" << di.data.length() << "]"; +} + +//////////////////////////////////////////////////////////////////////////// +// MockAzureFileSystem implementation + +class MockAzureFileSystem::Impl { + public: + TimePoint current_time; + MemoryPool* pool; + + // The root directory + Entry root; + std::mutex mutex; + + Impl(TimePoint current_time, MemoryPool* pool) + : current_time(current_time), pool(pool), root(Directory("", "", current_time)) {} + + std::unique_lock lock_guard() { + return std::unique_lock(mutex); + } + + Directory& RootDir() { return root.as_dir(); } + + template + Entry* FindEntry(It it, It end, size_t* nconsumed) { + size_t consumed = 0; + Entry* entry = &root; + + for (; it != end; ++it) { + const std::string& part = *it; + DCHECK(entry->is_dir()); + Entry* child = entry->as_dir().Find(part); + if (child == nullptr) { + // Partial find only + break; + } + ++consumed; + entry = child; + if (entry->is_file()) { + // Cannot go any further + break; + } + // Recurse + } + *nconsumed = consumed; + return entry; + } + + // Find an entry, allowing partial matching + Entry* FindEntry(const std::vector& parts, size_t* nconsumed) { + return FindEntry(parts.begin(), parts.end(), nconsumed); + } + + // Find an entry, only full matching allowed + Entry* FindEntry(const std::vector& parts) { + size_t consumed; + auto entry = FindEntry(parts, &consumed); + return (consumed == parts.size()) ? entry : nullptr; + } + + // Find the parent entry, only full matching allowed + Entry* FindParent(const std::vector& parts) { + if (parts.size() == 0) { + return nullptr; + } + size_t consumed; + auto entry = FindEntry(parts.begin(), --parts.end(), &consumed); + return (consumed == parts.size() - 1) ? entry : nullptr; + } + + bool CheckFile(const std::string& prefix, const Directory& dir, + const MockFileInfo& expected) { + std::string path = prefix + dir.name; + if (!path.empty()) { + path += "/"; + } + for (const auto& pair : dir.entries) { + Entry* child = pair.second.get(); + if (child->is_file()) { + auto& file = child->as_file(); + if ((path + file.name) == expected.full_path) { + if (util::string_view(file) == expected.data) { + return true; + } + } + } + } + bool res = false; + for (const auto& pair : dir.entries) { + Entry* child = pair.second.get(); + if (child->is_dir()) { + res = res || CheckFile(path, child->as_dir(), expected); + } + } + return res; + } + + Result> OpenOutputStream( + const std::string& path, bool append, + const std::shared_ptr& metadata) { + auto parts = SplitAbstractPath(path); + RETURN_NOT_OK(ValidateAbstractPathParts(parts)); + + Entry* parent = FindParent(parts); + if (parent == nullptr || !parent->is_dir()) { + return PathNotFound(path); + } + // Find the file in the parent dir, or create it + const auto& name = parts.back(); + Entry* child = parent->as_dir().Find(name); + File* file; + if (child == nullptr) { + child = new Entry(File(current_time, name, path)); + parent->as_dir().AssignEntry(name, std::unique_ptr(child)); + file = &child->as_file(); + file->path = path; + } else if (child->is_file()) { + file = &child->as_file(); + file->mtime = current_time; + file->path = path; + } else { + return NotAFile(path); + } + file->metadata = metadata; + auto ptr = std::make_shared(file, pool); + if (append && file->data) { + RETURN_NOT_OK(ptr->Write(file->data->data(), file->data->size())); + } + return ptr; + } + + Result> OpenInputReader(const std::string& path) { + auto parts = SplitAbstractPath(path); + RETURN_NOT_OK(ValidateAbstractPathParts(parts)); + + Entry* entry = FindEntry(parts); + if (entry == nullptr) { + return PathNotFound(path); + } + if (!entry->is_file()) { + return NotAFile(path); + } + return std::make_shared(entry->as_file()); + } + + // Create a container. Successful if container already exists. + Status CreateContainer(const std::string& container) { + auto parts = SplitAbstractPath(container); + size_t consumed; + Entry* entry = FindEntry(parts, &consumed); + if (consumed != 0) { + return Status::OK(); + } + std::unique_ptr child( + new Entry(Directory(container, container, current_time))); + child.get(); + bool inserted = entry->as_dir().CreateEntry(container, std::move(child)); + DCHECK(inserted); + return Status::OK(); + } + + // Tests to see if a container exists + Result ContainerExists(const std::string& container) { + auto parts = SplitAbstractPath(container); + size_t consumed; + FindEntry(parts, &consumed); + if (consumed != 0) { + return true; + } + return false; + } + + Result DirExists(const std::string& s) { + auto parts = SplitAbstractPath(s); + Entry* entry = FindEntry(parts); + if (entry == nullptr || !entry->is_dir()) { + return false; + } + return true; + } + + Result FileExists(const std::string& s) { + auto parts = SplitAbstractPath(s); + Entry* entry = FindEntry(parts); + if (entry == nullptr || !entry->is_file()) { + return false; + } + return true; + } + + Status CreateEmptyDir(const std::string& container, + const std::vector& path) { + std::vector parts = path; + parts.insert(parts.begin(), container); + size_t consumed; + Entry* entry = FindEntry(parts, &consumed); + if (!entry->is_dir()) { + auto file_path = JoinAbstractPath(parts.begin(), parts.begin() + consumed); + return Status::IOError("Cannot create directory: ", "ancestor '", file_path, + "' is not a directory"); + } + std::string str; + for (size_t i = 0; i < consumed; ++i) { + str += parts[i]; + if ((i + 1) < consumed) { + str += "/"; + } + } + for (size_t i = consumed; i < parts.size(); ++i) { + const auto& name = parts[i]; + str += "/"; + str += name; + std::unique_ptr child(new Entry(Directory(name, str, current_time))); + Entry* child_ptr = child.get(); + bool inserted = entry->as_dir().CreateEntry(name, std::move(child)); + DCHECK(inserted); + entry = child_ptr; + } + return Status::OK(); + } + + Status DeleteContainer(const std::string& container, Directory& rootDir) { + auto child = rootDir.Find(container); + if (child == nullptr) { + return Status::OK(); + } + bool deleted = rootDir.DeleteEntry(container); + DCHECK(deleted); + return Status::OK(); + } + + Status DeleteDir(const std::string& container, const std::vector& path, + const std::string& path_to_dir) { + std::vector parts = path; + parts.insert(parts.begin(), container); + + Entry* parent = FindParent(parts); + if (parent == nullptr || !parent->is_dir()) { + return PathNotFound(path_to_dir); + } + Directory& parent_dir = parent->as_dir(); + auto child = parent_dir.Find(parts.back()); + if (child == nullptr) { + return PathNotFound(path_to_dir); + } + if (!child->is_dir()) { + return NotADir(path_to_dir); + } + + bool deleted = parent_dir.DeleteEntry(parts.back()); + DCHECK(deleted); + return Status::OK(); + } + + Status DeleteFile(const std::string& container, const std::vector& path, + const std::string& path_to_file) { + if (path.empty()) { + return Status::IOError("Cannot delete File, Invalid File Path"); + } + std::vector parts = path; + parts.insert(parts.begin(), container); + + Entry* parent = FindParent(parts); + if (parent == nullptr || !parent->is_dir()) { + return PathNotFound(path_to_file); + } + Directory& parent_dir = parent->as_dir(); + auto child = parent_dir.Find(parts.back()); + if (child == nullptr) { + return PathNotFound(path_to_file); + } + if (!child->is_file()) { + return NotAFile(path_to_file); + } + bool deleted = parent_dir.DeleteEntry(parts.back()); + DCHECK(deleted); + return Status::OK(); + } + + Status ListPaths(const std::string& container, const std::string& path, + std::vector* childrenDirs, + std::vector* childrenFiles, + const bool allow_not_found = false) { + auto parts = SplitAbstractPath(path); + parts.insert(parts.begin(), container); + Entry* entry = FindEntry(parts); + Directory& base_dir = entry->as_dir(); + try { + for (const auto& pair : base_dir.entries) { + Entry* child = pair.second.get(); + if (child->is_dir()) { + childrenDirs->push_back(child->GetInfo().path()); + } + if (child->is_file()) { + childrenFiles->push_back(child->GetInfo().path()); + } + } + } catch (std::exception const& e) { + if (!allow_not_found) { + return Status::IOError("Path does not exists"); + } + } + return Status::OK(); + } + + Status Walk(const FileSelector& select, const std::string& container, + const std::string& path, int nesting_depth, std::vector* out) { + std::vector childrenDirs; + std::vector childrenFiles; + + Status st = + ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); + if (!st.ok()) { + return st; + } + + for (const auto& childFile : childrenFiles) { + FileInfo info; + auto parts = SplitAbstractPath(childFile); + Entry* entry = FindEntry(parts); + info = entry->GetInfo(); + out->push_back(std::move(info)); + } + for (const auto& childDir : childrenDirs) { + FileInfo info; + auto parts = SplitAbstractPath(childDir); + Entry* entry = FindEntry(parts); + if (entry == nullptr) { + return Status::OK(); + } + info = entry->GetInfo(); + out->push_back(std::move(info)); + if (select.recursive && nesting_depth < select.max_recursion) { + const auto src = internal::RemoveTrailingSlash(childDir); + auto first_sep = src.find_first_of("/"); + std::string s = std::string(src.substr(first_sep + 1)); + RETURN_NOT_OK(Walk(select, container, s, nesting_depth + 1, out)); + } + } + return Status::OK(); + } + + Status DeleteDirContents(const std::string& container, const std::string& path, + const std::vector& path_to_file_parts) { + std::vector childrenDirs; + std::vector childrenFiles; + + Status st = ListPaths(container, path, &childrenDirs, &childrenFiles); + if (!st.ok()) { + return st; + } + for (const auto& childFile : childrenFiles) { + ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); + DeleteFile(filePath.container, filePath.path_to_file_parts, filePath.full_path); + } + for (const auto& childDir : childrenDirs) { + ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); + DeleteDir(dirPath.container, dirPath.path_to_file_parts, dirPath.full_path); + } + return Status::OK(); + } + + Result> ListContainers(const Directory& base_dir) { + std::vector containers; + for (const auto& pair : base_dir.entries) { + Entry* child = pair.second.get(); + containers.push_back(child->GetInfo().path()); + } + return containers; + } +}; + +MockAzureFileSystem::~MockAzureFileSystem() = default; + +MockAzureFileSystem::MockAzureFileSystem(TimePoint current_time, + const io::IOContext& io_context) { + impl_ = std::unique_ptr(new Impl(current_time, io_context.pool())); +} + +bool MockAzureFileSystem::Equals(const FileSystem& other) const { return this == &other; } + +Status MockAzureFileSystem::CreateDir(const std::string& s, bool recursive) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty()) { + return Status::IOError("Cannot create directory, root path given"); + } + if ((impl_->FileExists(path.full_path)).ValueOrDie()) { + return Status::IOError("Cannot create directory, file exists at path"); + } + if (path.path_to_file.empty()) { + // Create container + return impl_->CreateContainer(path.container); + } + if (recursive) { + // Ensure container exists + ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); + if (!container_exists) { + RETURN_NOT_OK(impl_->CreateContainer(path.container)); + } + std::vector parent_path_to_file; + + for (const auto& part : path.path_to_file_parts) { + parent_path_to_file.push_back(part); + RETURN_NOT_OK(impl_->CreateEmptyDir(path.container, parent_path_to_file)); + } + return Status::OK(); + } else { + // Check parent dir exists + if (path.has_parent()) { + AzurePath parent_path = path.parent(); + if (parent_path.path_to_file.empty()) { + auto exists = impl_->ContainerExists(parent_path.container); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot create directory '", path.full_path, + "': parent directory does not exist"); + } + } else { + auto exists = impl_->DirExists(parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot create directory '", path.full_path, + "': parent directory does not exist"); + } + } + } + return impl_->CreateEmptyDir(path.container, path.path_to_file_parts); + } +} + +Status MockAzureFileSystem::DeleteDir(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + if (path.empty()) { + return Status::NotImplemented("Cannot delete all Azure Containers"); + } + if (path.path_to_file.empty()) { + return impl_->DeleteContainer(path.container, impl_->RootDir()); + } + if ((impl_->FileExists(path.full_path)).ValueOrDie()) { + return Status::IOError("Cannot delete directory, file exists at path"); + } + return impl_->DeleteDir(path.container, path.path_to_file_parts, path.full_path); +} + +Status MockAzureFileSystem::DeleteDirContents(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty()) { + return Status::IOError("Invalid path provided"); + } + + if (path.path_to_file.empty() && + !(impl_->ContainerExists(path.container).ValueOrDie())) { + return Status::IOError("Invalid path provided1"); + } + + if (impl_->FileExists(path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided2"); + } + + if (!(path.path_to_file.empty()) && !(impl_->DirExists(path.full_path).ValueOrDie())) { + return Status::IOError("Invalid path provided3"); + } + + return impl_->DeleteDirContents(path.container, path.path_to_file, + path.path_to_file_parts); +} + +Status MockAzureFileSystem::DeleteRootDirContents() { + auto guard = impl_->lock_guard(); + + impl_->RootDir().entries.clear(); + return Status::OK(); +} + +Status MockAzureFileSystem::DeleteFile(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + return impl_->DeleteFile(path.container, path.path_to_file_parts, path.full_path); +} + +Result MockAzureFileSystem::GetFileInfo(const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + FileInfo info; + info.set_path(s); + + if (path.empty()) { + // It's the root path "" + info.set_type(FileType::Directory); + return info; + } else if (path.path_to_file.empty()) { + // It's a container + ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); + if (!container_exists) { + info.set_type(FileType::NotFound); + return info; + } + info.set_type(FileType::Directory); + return info; + } else { + // It's an object + ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(path.full_path)); + if (file_exists) { + // "File" object found + std::vector parts = path.path_to_file_parts; + parts.insert(parts.begin(), path.container); + Entry* entry = impl_->FindEntry(parts); + info = entry->GetInfo(); + info.set_path(s); + return info; + } + // Not found => perhaps it's a "directory" + auto is_dir = impl_->DirExists(path.full_path); + if (is_dir.ValueOrDie()) { + info.set_type(FileType::Directory); + } else { + info.set_type(FileType::NotFound); + } + return info; + } +} + +Result MockAzureFileSystem::GetFileInfo(const FileSelector& select) { + ARROW_ASSIGN_OR_RAISE(auto base_path, AzurePath::FromString(select.base_dir)); + + FileInfoVector results; + + if (base_path.empty()) { + // List all containers + ARROW_ASSIGN_OR_RAISE(auto containers, impl_->ListContainers(impl_->RootDir())); + for (const auto& container : containers) { + FileInfo info; + auto parts = SplitAbstractPath(container); + Entry* entry = impl_->FindEntry(parts); + info = entry->GetInfo(); + info.set_path(container); + results.push_back(std::move(info)); + if (select.recursive) { + RETURN_NOT_OK(impl_->Walk(select, container, "", 0, &results)); + } + } + return results; + } + + if (base_path.path_to_file.empty() && + !(impl_->ContainerExists(base_path.container).ValueOrDie())) { + if (!select.allow_not_found) { + return Status::IOError("Invalid path provided"); + } + return results; + } + + if (impl_->FileExists(base_path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + + if (!(base_path.path_to_file.empty()) && + !(impl_->DirExists(base_path.full_path).ValueOrDie())) { + if (!select.allow_not_found) { + return Status::IOError("Invalid path provided"); + } + return results; + } + + // Nominal case -> walk a single container + RETURN_NOT_OK( + impl_->Walk(select, base_path.container, base_path.path_to_file, 0, &results)); + return results; +} + +namespace { + +// Helper for binary operations (move, copy) +struct BinaryOp { + std::vector src_parts; + std::vector dest_parts; + Directory& src_dir; + Directory& dest_dir; + std::string src_name; + std::string dest_name; + Entry* src_entry; + Entry* dest_entry; + + template + static Status Run(MockAzureFileSystem::Impl* impl, const std::string& src, + const std::string& dest, OpFunc&& op_func) { + RETURN_NOT_OK(ValidatePath(src)); + RETURN_NOT_OK(ValidatePath(dest)); + auto src_parts = SplitAbstractPath(src); + auto dest_parts = SplitAbstractPath(dest); + RETURN_NOT_OK(ValidateAbstractPathParts(src_parts)); + RETURN_NOT_OK(ValidateAbstractPathParts(dest_parts)); + + auto guard = impl->lock_guard(); + + // Both source and destination must have valid parents + Entry* src_parent = impl->FindParent(src_parts); + if (src_parent == nullptr || !src_parent->is_dir()) { + return PathNotFound(src); + } + Entry* dest_parent = impl->FindParent(dest_parts); + if (dest_parent == nullptr || !dest_parent->is_dir()) { + return PathNotFound(dest); + } + Directory& src_dir = src_parent->as_dir(); + Directory& dest_dir = dest_parent->as_dir(); + DCHECK_GE(src_parts.size(), 1); + DCHECK_GE(dest_parts.size(), 1); + const auto& src_name = src_parts.back(); + const auto& dest_name = dest_parts.back(); + + BinaryOp op{std::move(src_parts), + std::move(dest_parts), + src_dir, + dest_dir, + src_name, + dest_name, + src_dir.Find(src_name), + dest_dir.Find(dest_name)}; + + return op_func(std::move(op)); + } +}; + +} // namespace + +Status MockAzureFileSystem::Move(const std::string& src, const std::string& dest) { + ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); + ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); + + if (src_path == dest_path) { + return Status::OK(); + } + return BinaryOp::Run(impl_.get(), src, dest, [&](const BinaryOp& op) -> Status { + if (op.src_entry == nullptr) { + return PathNotFound(src); + } + if (op.dest_entry != nullptr) { + if (op.dest_entry->is_file() && op.src_entry->is_dir()) { + return Status::IOError("Cannot replace destination '", dest, + "', which is a file, with directory '", src, "'"); + } + if (op.dest_entry->is_dir() && op.src_entry->is_file()) { + return Status::IOError("Cannot replace destination '", dest, + "', which is a directory, with file '", src, "'"); + } + if (op.dest_entry->is_dir() && op.dest_entry->as_dir().entries.size() != 0) { + return Status::IOError("Cannot replace destination '", dest, + "', destination not empty"); + } + } + if (op.src_parts.size() < op.dest_parts.size()) { + // Check if dest is a child of src + auto p = + std::mismatch(op.src_parts.begin(), op.src_parts.end(), op.dest_parts.begin()); + if (p.first == op.src_parts.end()) { + return Status::IOError("Cannot move '", src, "' into child path '", dest, "'"); + } + } + auto path = src_path.path_to_file_parts; + std::unique_ptr new_entry(new Entry(std::move(*op.src_entry))); + new_entry->SetName(op.dest_name); + bool deleted = op.src_dir.DeleteEntry(op.src_name); + DCHECK(deleted); + op.dest_dir.AssignEntry(op.dest_name, std::move(new_entry)); + return Status::OK(); + }); +} + +Status MockAzureFileSystem::CopyFile(const std::string& src, const std::string& dest) { + return BinaryOp::Run(impl_.get(), src, dest, [&](const BinaryOp& op) -> Status { + if (op.src_entry == nullptr) { + return PathNotFound(src); + } + if (!op.src_entry->is_file()) { + return NotAFile(src); + } + if (op.dest_parts.size() == 1) { + return Status::IOError("Cannot copy destination '", dest, + "', which is a container"); + } + if (op.dest_entry != nullptr) { + if (op.dest_entry->is_file() && op.src_entry->is_dir()) { + return Status::IOError("Cannot copy destination '", dest, + "', which is a file, with directory '", src, "'"); + } + if (op.dest_entry->is_dir() && op.src_entry->is_file()) { + return Status::IOError("Cannot copy destination '", dest, + "', which is a directory, with file '", src, "'"); + } + } + if (op.dest_entry != nullptr && op.dest_entry->is_dir()) { + return Status::IOError("Cannot replace destination '", dest, + "', which is a directory"); + } + + // Copy original entry, fix its name + std::unique_ptr new_entry(new Entry(File(op.src_entry->as_file()))); + new_entry->SetName(op.dest_name); + op.dest_dir.AssignEntry(op.dest_name, std::move(new_entry)); + return Status::OK(); + }); +} + +Result> MockAzureFileSystem::OpenInputStream( + const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + if (path.empty()) { + return Status::IOError("Invalid path provided"); + } + if (!(impl_->FileExists(s)).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + return impl_->OpenInputReader(s); +} + +Result> MockAzureFileSystem::OpenInputFile( + const std::string& s) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + if (path.empty()) { + return Status::IOError("Invalid path provided"); + } + if (!(impl_->FileExists(s)).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + return impl_->OpenInputReader(s); +} + +Result> MockAzureFileSystem::OpenOutputStream( + const std::string& s, const std::shared_ptr& metadata) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + if (path.empty() || path.path_to_file.empty()) { + return Status::IOError("Invalid path provided"); + } + if (impl_->DirExists(s).ValueOrDie()) { + return Status::IOError("Invalid path provided"); + } + if (path.has_parent()) { + AzurePath parent_path = path.parent(); + if (parent_path.path_to_file.empty()) { + if (!impl_->ContainerExists(parent_path.container).ValueOrDie()) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } else { + auto exists = impl_->DirExists(parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } + } + return impl_->OpenOutputStream(s, /*append=*/false, metadata); +} + +Result> MockAzureFileSystem::OpenAppendStream( + const std::string& s, const std::shared_ptr& metadata) { + ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + + return impl_->OpenOutputStream(s, /*append=*/true, metadata); +} + +bool MockAzureFileSystem::CheckFile(const MockFileInfo& expected) { + auto guard = impl_->lock_guard(); + + return impl_->CheckFile("", impl_->RootDir(), expected); +} + +Status MockAzureFileSystem::CreateFile(const std::string& path, + util::string_view contents, bool recursive) { + RETURN_NOT_OK(ValidatePath(path)); + auto parent = fs::internal::GetAbstractPathParent(path).first; + + if (parent != "") { + RETURN_NOT_OK(CreateDir(parent, recursive)); + } + + ARROW_ASSIGN_OR_RAISE(auto file, OpenOutputStream(path)); + RETURN_NOT_OK(file->Write(contents)); + return file->Close(); +} + +Result> MockAzureFileSystem::Make( + TimePoint current_time, const std::vector& infos) { + auto fs = std::make_shared(current_time); + for (const auto& info : infos) { + switch (info.type()) { + case FileType::Directory: + RETURN_NOT_OK(fs->CreateDir(info.path(), /*recursive*/ true)); + break; + case FileType::File: + RETURN_NOT_OK(fs->CreateFile(info.path(), "", /*recursive*/ true)); + break; + default: + break; + } + } + + return fs; +} + +FileInfoGenerator MockAsyncFileSystem::GetFileInfoGenerator(const FileSelector& select) { + auto maybe_infos = GetFileInfo(select); + if (maybe_infos.ok()) { + // Return the FileInfo entries one by one + const auto& infos = *maybe_infos; + std::vector chunks(infos.size()); + std::transform(infos.begin(), infos.end(), chunks.begin(), + [](const FileInfo& info) { return FileInfoVector{info}; }); + return MakeVectorGenerator(std::move(chunks)); + } else { + return MakeFailingGenerator(maybe_infos); + } +} + +} // namespace internal +} // namespace fs +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azurefs_mock.h b/cpp/src/arrow/filesystem/azurefs_mock.h new file mode 100644 index 0000000000000..f778ae04a70f5 --- /dev/null +++ b/cpp/src/arrow/filesystem/azurefs_mock.h @@ -0,0 +1,95 @@ +// 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 +#include +#include +#include + +#include "arrow/filesystem/mockfs.h" +#include "arrow/util/string_view.h" +#include "arrow/util/windows_fixup.h" + +namespace arrow { +namespace fs { +namespace internal { + +/// A mock FileSystem implementation that holds its contents in memory. +/// +/// Useful for validating the FileSystem API, writing conformance suite, +/// and bootstrapping FileSystem-based APIs. +class ARROW_EXPORT MockAzureFileSystem : public FileSystem { + public: + explicit MockAzureFileSystem(TimePoint current_time, + const io::IOContext& = io::default_io_context()); + ~MockAzureFileSystem() override; + + std::string type_name() const override { return "mock"; } + + bool Equals(const FileSystem& other) const override; + + // XXX It's not very practical to have to explicitly declare inheritance + // of default overrides. + using FileSystem::GetFileInfo; + Result GetFileInfo(const std::string& path) override; + Result> GetFileInfo(const FileSelector& select) override; + + Status CreateDir(const std::string& path, bool recursive = true) override; + + Status DeleteDir(const std::string& path) override; + Status DeleteDirContents(const std::string& path) override; + Status DeleteRootDirContents() override; + + Status DeleteFile(const std::string& path) override; + + Status Move(const std::string& src, const std::string& dest) override; + + Status CopyFile(const std::string& src, const std::string& dest) override; + + Result> OpenInputStream( + const std::string& path) override; + Result> OpenInputFile( + const std::string& path) override; + Result> OpenOutputStream( + const std::string& path, + const std::shared_ptr& metadata = {}) override; + Result> OpenAppendStream( + const std::string& path, + const std::shared_ptr& metadata = {}) override; + + bool CheckFile(const MockFileInfo& expected); + + // Create a File with a content from a string. + Status CreateFile(const std::string& path, util::string_view content, + bool recursive = true); + + // Create a MockAzureFileSystem out of (empty) FileInfo. The content of every + // file is empty and of size 0. All directories will be created recursively. + static Result> Make(TimePoint current_time, + const std::vector& infos); + + class Impl; + + protected: + std::unique_ptr impl_; +}; + +} // namespace internal +} // namespace fs +} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 539b415498080..1900fabfb4f21 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -15,15 +15,12 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/filesystem/azure/azurefs.h" +#include "arrow/filesystem/azurefs_mock.h" +#include #include #include -#include -#include -#include - #include "arrow/filesystem/test_util.h" #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" @@ -37,239 +34,35 @@ namespace arrow { using internal::Uri; namespace fs { +namespace internal { -class AzureEnvTestMixin { - public: - static AzureOptions options_; - static std::shared_ptr fs_; - static std::shared_ptr - gen2Client_; - static std::shared_ptr gen1Client_; - - AzureEnvTestMixin() {} - - // Test account info is fetched from environment variables. - // To run azure tests, first set these environment variables. - const std::string& GetAdlsGen2AccountName() { - static const std::string accountName = [&]() -> std::string { - return std::getenv("ADLS_GEN2_ACCOUNT_NAME"); - }(); - return accountName; - } - - const std::string& GetAdlsGen2AccountKey() { - static const std::string accountKey = [&]() -> std::string { - return std::getenv("ADLS_GEN2_ACCOUNT_KEY"); - }(); - return accountKey; - } - - const std::string& GetAdlsGen2ConnectionString() { - static const std::string connectionString = [&]() -> std::string { - return std::getenv("ADLS_GEN2_CONNECTION_STRING"); - }(); - return connectionString; - } - - const std::string& GetAdlsGen2SasUrl() { - static const std::string sasUrl = [&]() -> std::string { - return std::getenv("ADLS_GEN2_SASURL"); - }(); - return sasUrl; - } - - const std::string& GetAadTenantId() { - static const std::string tenantId = [&]() -> std::string { - return std::getenv("AAD_TENANT_ID"); - }(); - return tenantId; - } - - const std::string& GetAadClientId() { - static const std::string clientId = [&]() -> std::string { - return std::getenv("AAD_CLIENT_ID"); - }(); - return clientId; - } - - const std::string& GetAadClientSecret() { - static const std::string clientSecret = [&]() -> std::string { - return std::getenv("AAD_CLIENT_SECRET"); - }(); - return clientSecret; - } -}; - -AzureOptions AzureEnvTestMixin::options_; -std::shared_ptr AzureEnvTestMixin::fs_; -std::shared_ptr - AzureEnvTestMixin::gen2Client_; -std::shared_ptr AzureEnvTestMixin::gen1Client_; - -// A global test environment for setting up directories and files inside the specified -// container. - -class SetupEnvironment : public ::testing::Environment, public AzureEnvTestMixin { +class TestAzureFileSystem : public ::testing::Test { public: - bool isHeirarchialNamespaceEnabled() { - return AzureEnvTestMixin::gen1Client_->GetAccountInfo() - .Value.IsHierarchicalNamespaceEnabled; - } + TimePoint time_; + std::shared_ptr fs_; - void MakeFileSystem() { - const std::string& account_key = GetAdlsGen2AccountKey(); - const std::string& account_name = GetAdlsGen2AccountName(); - AzureEnvTestMixin::options_.ConfigureAccountKeyCredentials(account_name, account_key); - auto url = options_.account_dfs_url; - AzureEnvTestMixin::gen2Client_ = - std::make_shared( - url, options_.storage_credentials_provider); - AzureEnvTestMixin::gen1Client_ = - std::make_shared( - options_.account_blob_url, options_.storage_credentials_provider); - auto result = AzureBlobFileSystem::Make(options_); - if (!result.ok()) { - ARROW_LOG(INFO) << "AzureFileSystem::Make failed, err msg is " - << result.status().ToString(); - return; - } - AzureEnvTestMixin::fs_ = *result; - } + void MakeFileSystem() { fs_ = std::make_shared(time_); } void SetUp() override { - { - auto fileSystemClient = - AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container"); - fileSystemClient.CreateIfNotExists(); - fileSystemClient = - AzureEnvTestMixin::gen2Client_->GetFileSystemClient("empty-container"); - fileSystemClient.CreateIfNotExists(); - } - { - if (isHeirarchialNamespaceEnabled()) { - auto directoryClient = - AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container") - .GetDirectoryClient("emptydir"); - directoryClient.CreateIfNotExists(); - directoryClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient("container") - .GetDirectoryClient("somedir"); - directoryClient.CreateIfNotExists(); - directoryClient = directoryClient.GetSubdirectoryClient("subdir"); - directoryClient.CreateIfNotExists(); - auto fileClient = directoryClient.GetFileClient("subfile"); - fileClient.CreateIfNotExists(); - std::string s = "sub data"; - fileClient.UploadFrom( - const_cast(reinterpret_cast(&s[0])), s.size()); - fileClient = - gen2Client_->GetFileSystemClient("container").GetFileClient("somefile"); - fileClient.CreateIfNotExists(); - s = "some data"; - fileClient.UploadFrom( - const_cast(reinterpret_cast(&s[0])), s.size()); - } else { - auto fc = std::make_shared( - options_.account_blob_url + "container/somefile", - options_.storage_credentials_provider); - std::string s = "some data"; - fc->UploadFrom(const_cast(reinterpret_cast(&s[0])), - s.size()); - } - } + time_ = TimePoint(TimePoint::duration(42)); + MakeFileSystem(); + fs_->CreateDir("container"); + fs_->CreateDir("empty-container"); + fs_->CreateDir("container2/newdir"); + fs_->CreateDir("container/emptydir"); + fs_->CreateDir("container/somedir"); + fs_->CreateDir("container/somedir/subdir"); + CreateFile(fs_.get(), "container/somedir/subdir/subfile", "sub data"); + CreateFile(fs_.get(), "container/somefile", "some data"); } - void TearDown() override { - auto containers = AzureEnvTestMixin::gen2Client_->ListFileSystems(); - for (auto c : containers.FileSystems) { - auto fileSystemClient = AzureEnvTestMixin::gen2Client_->GetFileSystemClient(c.Name); - fileSystemClient.DeleteIfExists(); - } + bool CheckFile(const MockFileInfo& expected) { + return arrow::internal::checked_pointer_cast(fs_)->CheckFile( + expected); } }; -class TestAzureFileSystem : public ::testing::Test, public AzureEnvTestMixin { - public: - void AssertObjectContents( - Azure::Storage::Files::DataLake::DataLakeServiceClient* client, - const std::string& container, const std::string& path_to_file, - const std::string& expected) { - auto pathClient_ = - std::make_shared( - client->GetUrl() + "/" + container + "/" + path_to_file, - options_.storage_credentials_provider); - auto size = pathClient_->GetProperties().Value.FileSize; - auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); - Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; - Azure::Core::Http::HttpRange range; - range.Offset = 0; - range.Length = size; - downloadOptions.Range = Azure::Nullable(range); - auto fileClient_ = - std::make_shared( - client->GetUrl() + "/" + container + "/" + path_to_file, - options_.storage_credentials_provider); - auto result = fileClient_ - ->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), - size, downloadOptions) - .Value; - buf->get()->Equals( - Buffer(const_cast(reinterpret_cast(&expected[0])), - expected.size())); - } -}; - -TEST(TestAzureFSOptions, FromUri) { - AzureOptions options; - Uri uri; - - // Public container - ASSERT_OK(uri.Parse("https://testcontainer.dfs.core.windows.net/")); - ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Anonymous); - ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); - - // Sas Token - ASSERT_OK(uri.Parse("https://testcontainer.blob.core.windows.net/?dummy_sas_token")); - ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); - ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); - ASSERT_EQ(options.sas_token, "?dummy_sas_token"); -} - -TEST_F(TestAzureFileSystem, FromAccountKey) { - AzureOptions options; - options = AzureOptions::FromAccountKey(this->GetAdlsGen2AccountKey(), - this->GetAdlsGen2AccountName()); - ASSERT_EQ(options.credentials_kind, - arrow::fs::AzureCredentialsKind::StorageCredentials); - ASSERT_NE(options.storage_credentials_provider, nullptr); -} - -TEST_F(TestAzureFileSystem, FromConnectionString) { - AzureOptions options; - options = AzureOptions::FromConnectionString(this->GetAdlsGen2ConnectionString()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ConnectionString); - ASSERT_NE(options.connection_string, ""); -} - -TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { - AzureOptions options; - options = AzureOptions::FromServicePrincipleCredential( - this->GetAdlsGen2AccountName(), this->GetAadTenantId(), this->GetAadClientId(), - this->GetAadClientSecret()); - ASSERT_EQ(options.credentials_kind, - arrow::fs::AzureCredentialsKind ::ServicePrincipleCredentials); - ASSERT_NE(options.service_principle_credentials_provider, nullptr); -} - -TEST_F(TestAzureFileSystem, FromSas) { - AzureOptions options; - options = AzureOptions::FromSas(this->GetAdlsGen2SasUrl()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); - ASSERT_NE(options.sas_token, ""); -} - -TEST_F(TestAzureFileSystem, CreateDirBlobStorage) { +TEST_F(TestAzureFileSystem, CreateDir) { // New container AssertFileInfo(fs_.get(), "container3", FileType::NotFound); ASSERT_OK(fs_->CreateDir("container3")); @@ -281,367 +74,6 @@ TEST_F(TestAzureFileSystem, CreateDirBlobStorage) { ASSERT_RAISES(IOError, fs_->CreateDir("")); - // Existing "file", should fail - ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); - - // directory, false - ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", false)); - - // directory, true - ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", true)); -} - -TEST_F(TestAzureFileSystem, DeleteDirBlobStorage) { - FileSelector select; - select.base_dir = "container4"; - std::vector infos; - - // Container - ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - - // Nonexistent Container - ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - - // root - ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); - - // C/F - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); - - // C/NF - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile19")); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); -} - -TEST_F(TestAzureFileSystem, DeleteFileBlobStorage) { - FileSelector select; - select.base_dir = "container4"; - std::vector infos; - - // Container - ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); - - // Nonexistent Container - ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); - - // root - ASSERT_RAISES(IOError, fs_->DeleteFile("")); - - // C/F - ASSERT_OK(fs_->DeleteFile("container/somefile")); - - // C/NF - ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); - - // C/D/D - ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); -} - -TEST_F(TestAzureFileSystem, GetFileInfoBlobStorage) { - // Containers - AssertFileInfo(fs_.get(), "container", FileType::Directory); - AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); - - AssertFileInfo(fs_.get(), "", FileType::Directory); - - auto res = fs_->OpenOutputStream("container/base.txt"); - res->get()->Write("Changed the data"); - - // "Files" - AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); - AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); - - // "Directories" - AssertFileInfo(fs_.get(), "container/somedir45/subdir", FileType::NotFound); - AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); -} - -TEST_F(TestAzureFileSystem, GetFileInfoSelectorBlobStorage) { - FileSelector select; - std::vector infos; - - // Non-empty container - select.base_dir = "container"; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 1); - - // Nonexistent container - select.base_dir = "nonexistent-container"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; - - // Root dir - select.base_dir = ""; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 2); - - // C/F - select.base_dir = "container/base.txt"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - - // C/ND/D - select.base_dir = "container/ahsh/agsg"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; - - // NC/D - select.base_dir = "nonexistent-container/agshhs"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; -} - -TEST_F(TestAzureFileSystem, MoveBlobStorage) { - ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); - ASSERT_RAISES(IOError, - fs_->Move("container/somedir/subdir", "container/newdir/newsub")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "containerqw/ghdj")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("Changed the data"); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); - ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/ahsh/gssjd")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "containerqw/ghdj")); - ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/gshh")); -} - -TEST_F(TestAzureFileSystem, CopyFileBlobStorage) { - ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container23/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "container27/hshj")); - - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("Changed the data"); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container/sjdj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir/subdir")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); -} - -TEST_F(TestAzureFileSystem, OpenInputStreamBlobStorage) { - std::shared_ptr stream; - std::shared_ptr buf; - - ASSERT_RAISES(IOError, fs_->OpenInputStream("container")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container263")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/sjdjd")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container526/somedir")); - - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("some data"); - - // "Files" - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(2)); - AssertBufferEqual(*buf, "so"); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); - AssertBufferEqual(*buf, "me da"); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); - AssertBufferEqual(*buf, "ta"); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); - AssertBufferEqual(*buf, ""); -} - -TEST_F(TestAzureFileSystem, OpenInputFileBlobStorage) { - std::shared_ptr file; - std::shared_ptr buf; - - ASSERT_RAISES(IOError, fs_->OpenInputFile("container")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container263")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/sjdjd")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container526/somedir")); - - // "Files" - ASSERT_OK_AND_ASSIGN(file, fs_->OpenInputFile("container/somefile")); - ASSERT_OK_AND_EQ(9, file->GetSize()); - ASSERT_OK_AND_ASSIGN(buf, file->Read(4)); - AssertBufferEqual(*buf, "some"); - ASSERT_OK_AND_EQ(9, file->GetSize()); - ASSERT_OK_AND_EQ(4, file->Tell()); - - ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(2, 5)); - AssertBufferEqual(*buf, "me da"); - ASSERT_OK_AND_EQ(4, file->Tell()); - ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(5, 20)); - AssertBufferEqual(*buf, "data"); - ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(9, 20)); - AssertBufferEqual(*buf, ""); - - char result[10]; - ASSERT_OK_AND_EQ(5, file->ReadAt(2, 5, &result)); - ASSERT_OK_AND_EQ(4, file->ReadAt(5, 20, &result)); - ASSERT_OK_AND_EQ(0, file->ReadAt(9, 0, &result)); - - // Reading past end of file - ASSERT_RAISES(IOError, file->ReadAt(10, 20)); - - ASSERT_OK(file->Seek(5)); - ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); - AssertBufferEqual(*buf, "da"); - ASSERT_OK(file->Seek(9)); - ASSERT_OK_AND_ASSIGN(buf, file->Read(2)); - AssertBufferEqual(*buf, ""); - // Seeking past end of file - ASSERT_RAISES(IOError, file->Seek(10)); -} - -TEST_F(TestAzureFileSystem, OpenOutputStreamBlobStorage) { - std::shared_ptr stream; - - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container263")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container526/somedir")); - - // Create new empty file - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); - - // Create new file with 1 small write - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); - ASSERT_OK(stream->Write("some data")); - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); - - // Create new file with 3 small writes - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); - ASSERT_OK(stream->Write("some ")); - ASSERT_OK(stream->Write("")); - ASSERT_OK(stream->Write("new data")); - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); - - // Create new file with some large writes - std::string s1, s2, s3, s4, s5, expected; - s1 = random_string(6000000, /*seed =*/42); // More than the 5 MB minimum part upload - s2 = "xxx"; - s3 = random_string(6000000, 43); - s4 = "zzz"; - s5 = random_string(600000, 44); - expected = s1 + s2 + s3 + s4 + s5; - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile4")); - for (auto input : {s1, s2, s3, s4, s5}) { - ASSERT_OK(stream->Write(input)); - // Clobber source contents. This shouldn't reflect in the data written. - input.front() = 'x'; - input.back() = 'x'; - } - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); - - // Overwrite - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); - ASSERT_OK(stream->Write("overwritten data")); - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); - - // Overwrite and make empty - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); -} - -TEST_F(TestAzureFileSystem, DeleteDirContentsBlobStorage) { - FileSelector select; - select.base_dir = "container4/newdir"; - std::vector infos; - - // Container - fs_->CreateDir("container4"); - ASSERT_OK(fs_->DeleteDirContents("container4")); - AssertFileInfo(fs_.get(), "container4", FileType::Directory); - - // Nonexistent Container - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - - // root - ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - - // C/F - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); - AssertFileInfo(fs_.get(), "container/somefile", FileType::File); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3/somedir")); -} - -TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { - // New container - auto op = fs_->options(); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - ASSERT_OK(fs_->CreateDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::Directory); - - // Existing container - ASSERT_OK(fs_->CreateDir("container")); - AssertFileInfo(fs_.get(), "container", FileType::Directory); - - ASSERT_RAISES(IOError, fs_->CreateDir("")); - // New "directory", true AssertFileInfo(fs_.get(), "container/newdir", FileType::NotFound); ASSERT_OK(fs_->CreateDir("container/newdir", true)); @@ -695,14 +127,14 @@ TEST_F(TestAzureFileSystem, CreateDirAdlsGen2) { AssertFileInfo(fs_.get(), "container4/newdir", FileType::Directory); } -TEST_F(TestAzureFileSystem, DeleteDirAdlsGen2) { +TEST_F(TestAzureFileSystem, DeleteDir) { FileSelector select; - select.base_dir = "container4"; + select.base_dir = "container2"; std::vector infos; // Container - ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + ASSERT_OK(fs_->DeleteDir("empty-container")); + AssertFileInfo(fs_.get(), "empty-container", FileType::NotFound); // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); @@ -712,39 +144,39 @@ TEST_F(TestAzureFileSystem, DeleteDirAdlsGen2) { ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); // C/D - ASSERT_OK(fs_->DeleteDir("container4/newdir")); - // ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - // ASSERT_EQ(infos.size(), 0); + ASSERT_OK(fs_->DeleteDir("container2/newdir")); + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); // C/ND - AssertFileInfo(fs_.get(), "container4/newdir", FileType::NotFound); - ASSERT_RAISES(IOError, fs_->DeleteDir("container4/newdir")); - // ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - // ASSERT_EQ(infos.size(), 0); + AssertFileInfo(fs_.get(), "container2/newdir1", FileType::NotFound); + ASSERT_RAISES(IOError, fs_->DeleteDir("container2/newdir1")); + ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_EQ(infos.size(), 0); // C/F ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); // C/D/D - ASSERT_OK(fs_->DeleteDir("container/newdir/newsub")); + ASSERT_OK(fs_->DeleteDir("container/somedir/subdir")); // C/D/F - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir/base.txt")); + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir/subdir/subfile")); // C/ND/D ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); + ASSERT_RAISES(IOError, fs_->DeleteDir("container6/somedir")); } -TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { +TEST_F(TestAzureFileSystem, DeleteFile) { FileSelector select; - select.base_dir = "container4"; + select.base_dir = "container2"; std::vector infos; // Container - ASSERT_RAISES(IOError, fs_->DeleteFile("container4")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container2")); // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); @@ -753,10 +185,10 @@ TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { ASSERT_RAISES(IOError, fs_->DeleteFile("")); // C/D - ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptyDir")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptydir")); // C/ND - ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptyDir1")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptydir1")); // C/F ASSERT_OK(fs_->DeleteFile("container/somefile")); @@ -767,21 +199,18 @@ TEST_F(TestAzureFileSystem, DeleteFileAdlsGen2) { // C/D/D ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); - auto res = fs_->OpenOutputStream("container/somedir/base.txt"); - res->get()->Write("Changed the data"); - // C/D/F - AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); - ASSERT_OK(fs_->DeleteFile("container/somedir/base.txt")); + AssertFileInfo(fs_.get(), "container/somedir/subdir/subfile", FileType::File); + ASSERT_OK(fs_->DeleteFile("container/somedir/subdir/subfile")); // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir3/base")); // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container7/somedir")); } -TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { +TEST_F(TestAzureFileSystem, GetFileInfo) { // Containers AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); @@ -792,12 +221,9 @@ TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { AssertFileInfo(fs_.get(), "container/emptydir", FileType::Directory); AssertFileInfo(fs_.get(), "container/emptydir1", FileType::NotFound); - auto res = fs_->OpenOutputStream("container/base.txt"); - res->get()->Write("Changed the data"); - // "Files" - AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); - AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); + AssertFileInfo(fs_.get(), "container/somefile", FileType::File); + AssertFileInfo(fs_.get(), "container/somefile1", FileType::NotFound); // "Directories" AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); @@ -808,14 +234,14 @@ TEST_F(TestAzureFileSystem, GetFileInfoAdlsGen2) { AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); } -TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { +TEST_F(TestAzureFileSystem, GetFileInfoSelector) { FileSelector select; std::vector infos; // Non-empty container select.base_dir = "container"; ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 5); + ASSERT_EQ(infos.size(), 3); // Nonexistent container select.base_dir = "nonexistent-container"; @@ -844,7 +270,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { select.allow_not_found = false; // C/F - select.base_dir = "container/base.txt"; + select.base_dir = "container/somefile"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); // C/D/D @@ -873,7 +299,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelectorAdlsGen2) { select.allow_not_found = false; } -TEST_F(TestAzureFileSystem, MoveAdlsGen2) { +TEST_F(TestAzureFileSystem, Move) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); fs_->CreateDir("container/newdir/newsub/newsubsub", true); ASSERT_RAISES(IOError, @@ -881,20 +307,17 @@ TEST_F(TestAzureFileSystem, MoveAdlsGen2) { ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); ASSERT_OK(fs_->Move("container/emptydir", "container/emptydir1")); ASSERT_OK(fs_->Move("container/emptydir1", "container/emptydir")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/somefile")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); - ASSERT_OK(fs_->Move("container/emptydir", "containerqw/ghdj")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("Changed the data"); - ASSERT_OK(fs_->Move("container/base.txt", "container/somefile")); + ASSERT_OK(fs_->Move("container/somedir/subdir/subfile", "container/somefile")); ASSERT_OK(fs_->Move("container/somefile", "container/base.txt")); ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/ahsh/gssjd")); ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "containerqw/ghdj")); ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/gshh")); } -TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { +TEST_F(TestAzureFileSystem, CopyFile) { // "File" ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); @@ -912,11 +335,8 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container27/hshj")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "container27/hshj")); - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("Changed the data"); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile")); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somedir/subdir/subfile")); + ASSERT_OK(fs_->CopyFile("container/somefile", "container/somedir/subdir/subfile")); + ASSERT_OK(fs_->CopyFile("container/somefile", "container/somefile3")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); @@ -936,7 +356,7 @@ TEST_F(TestAzureFileSystem, CopyFileAdlsGen2) { ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); } -TEST_F(TestAzureFileSystem, OpenInputStreamGen2) { +TEST_F(TestAzureFileSystem, OpenInputStream) { std::shared_ptr stream; std::shared_ptr buf; @@ -949,9 +369,6 @@ TEST_F(TestAzureFileSystem, OpenInputStreamGen2) { ASSERT_RAISES(IOError, fs_->OpenInputStream("container/shjdj/subdir")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container526/somedir")); - auto res = fs_->OpenOutputStream("container/somefile"); - res->get()->Write("some data"); - // "Files" ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); ASSERT_OK_AND_ASSIGN(buf, stream->Read(2)); @@ -963,10 +380,9 @@ TEST_F(TestAzureFileSystem, OpenInputStreamGen2) { ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); AssertBufferEqual(*buf, ""); - res = fs_->OpenOutputStream("container/somedir/subdir/subfile"); - res->get()->Write("sub data"); + CreateFile(fs_.get(), "container/subfile", "sub data"); - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somedir/subdir/subfile")); + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/subfile")); ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); AssertBufferEqual(*buf, "sub data"); ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); @@ -974,7 +390,7 @@ TEST_F(TestAzureFileSystem, OpenInputStreamGen2) { ASSERT_OK(stream->Close()); } -TEST_F(TestAzureFileSystem, OpenInputFileGen2) { +TEST_F(TestAzureFileSystem, OpenInputFile) { std::shared_ptr file; std::shared_ptr buf; @@ -1021,7 +437,7 @@ TEST_F(TestAzureFileSystem, OpenInputFileGen2) { ASSERT_RAISES(IOError, file->Seek(10)); } -TEST_F(TestAzureFileSystem, OpenOutputStreamGen2) { +TEST_F(TestAzureFileSystem, OpenOutputStream) { std::shared_ptr stream; ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); @@ -1035,13 +451,13 @@ TEST_F(TestAzureFileSystem, OpenOutputStreamGen2) { // Create new empty file ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + ASSERT_TRUE(CheckFile({"container/newfile1", time_, ""})); // Create new file with 1 small write ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); ASSERT_OK(stream->Write("some data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); + ASSERT_TRUE(CheckFile({"container/newfile2", time_, "some data"})); // Create new file with 3 small writes ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); @@ -1049,7 +465,7 @@ TEST_F(TestAzureFileSystem, OpenOutputStreamGen2) { ASSERT_OK(stream->Write("")); ASSERT_OK(stream->Write("new data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); + ASSERT_TRUE(CheckFile({"container/newfile3", time_, "some new data"})); // Create new file with some large writes std::string s1, s2, s3, s4, s5, expected; @@ -1067,28 +483,28 @@ TEST_F(TestAzureFileSystem, OpenOutputStreamGen2) { input.back() = 'x'; } ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); + ASSERT_TRUE(CheckFile({"container/newfile4", time_, expected})); // Overwrite ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Write("overwritten data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); + ASSERT_TRUE(CheckFile({"container/newfile1", time_, "overwritten data"})); // Overwrite and make empty ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + ASSERT_TRUE(CheckFile({"container/newfile1", time_, ""})); } -TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { +TEST_F(TestAzureFileSystem, DeleteDirContents) { FileSelector select; - select.base_dir = "container4/newdir"; + select.base_dir = "container2/newdir"; std::vector infos; // Container - ASSERT_OK(fs_->DeleteDirContents("container4")); - AssertFileInfo(fs_.get(), "container4", FileType::Directory); + ASSERT_OK(fs_->DeleteDirContents("container2")); + AssertFileInfo(fs_.get(), "container2", FileType::Directory); // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); @@ -1096,16 +512,16 @@ TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - fs_->CreateDir("container4/newdir/subdir", true); + fs_->CreateDir("container2/newdir/subdir", true); // C/D - ASSERT_OK(fs_->DeleteDirContents("container4/newdir")); + ASSERT_OK(fs_->DeleteDirContents("container2/newdir")); ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); ASSERT_EQ(infos.size(), 0); // C/ND - AssertFileInfo(fs_.get(), "container4/newdir1", FileType::NotFound); - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container4/newdir1")); + AssertFileInfo(fs_.get(), "container2/newdir1", FileType::NotFound); + ASSERT_RAISES(IOError, fs_->DeleteDirContents("container2/newdir1")); // C/F ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); @@ -1125,18 +541,6 @@ TEST_F(TestAzureFileSystem, DeleteDirContentsGen2) { ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3/somedir")); } +} // namespace internal } // namespace fs } // namespace arrow - -int main(int argc, char** argv) { - auto env = new arrow::fs::SetupEnvironment(); - env->MakeFileSystem(); - ::testing::AddGlobalTestEnvironment(env); - ::testing::InitGoogleTest(&argc, argv); - if (env->isHeirarchialNamespaceEnabled()) { - ::testing::GTEST_FLAG(filter) = "*From*:*Gen2"; - } else { - ::testing::GTEST_FLAG(filter) = "*From*:*BlobStorage"; - } - return RUN_ALL_TESTS(); -} From 5bd8210aace076280272db010061976f54b80796 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 26 May 2022 16:58:01 +0530 Subject: [PATCH 07/34] Turned -DARROW_AZURE=OFF in appveyor-cpp-build --- ci/appveyor-cpp-build.bat | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/appveyor-cpp-build.bat b/ci/appveyor-cpp-build.bat index e46947abad45a..fb16a0bf3c05b 100644 --- a/ci/appveyor-cpp-build.bat +++ b/ci/appveyor-cpp-build.bat @@ -108,7 +108,7 @@ cmake -G "%GENERATOR%" %CMAKE_ARGS% ^ -DARROW_PARQUET=ON ^ -DARROW_PYTHON=ON ^ -DARROW_S3=%ARROW_S3% ^ - -DARROW_AZURE=ON ^ + -DARROW_AZURE=OFF ^ -DARROW_SUBSTRAIT=ON ^ -DARROW_VERBOSE_THIRDPARTY_BUILD=OFF ^ -DARROW_WITH_BROTLI=ON ^ From eead6734bf674751b3bb492cb845dae35b15fdec Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 1 Jun 2022 05:30:42 +0530 Subject: [PATCH 08/34] Changed default C++ version --- ci/appveyor-cpp-build.bat | 2 +- cpp/cmake_modules/SetupCxxFlags.cmake | 10 ++++-- cpp/cmake_modules/ThirdpartyToolchain.cmake | 11 ++---- cpp/src/arrow/CMakeLists.txt | 7 ++++ cpp/src/arrow/filesystem/CMakeLists.txt | 29 +++------------ cpp/src/arrow/filesystem/azurefs.cc | 20 ++++++++--- cpp/src/arrow/filesystem/azurefs.h | 2 +- cpp/src/arrow/filesystem/azurefs_mock.cc | 39 +++++++-------------- cpp/src/arrow/filesystem/azurefs_mock.h | 2 +- 9 files changed, 52 insertions(+), 70 deletions(-) diff --git a/ci/appveyor-cpp-build.bat b/ci/appveyor-cpp-build.bat index fb16a0bf3c05b..652807a6ee14f 100644 --- a/ci/appveyor-cpp-build.bat +++ b/ci/appveyor-cpp-build.bat @@ -94,6 +94,7 @@ pushd cpp\build @rem and enable runtime assertions. cmake -G "%GENERATOR%" %CMAKE_ARGS% ^ + -DARROW_AZURE=OFF ^ -DARROW_BOOST_USE_SHARED=ON ^ -DARROW_BUILD_EXAMPLES=ON ^ -DARROW_BUILD_STATIC=OFF ^ @@ -108,7 +109,6 @@ cmake -G "%GENERATOR%" %CMAKE_ARGS% ^ -DARROW_PARQUET=ON ^ -DARROW_PYTHON=ON ^ -DARROW_S3=%ARROW_S3% ^ - -DARROW_AZURE=OFF ^ -DARROW_SUBSTRAIT=ON ^ -DARROW_VERBOSE_THIRDPARTY_BUILD=OFF ^ -DARROW_WITH_BROTLI=ON ^ diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index d3a2a1a2d256a..cdaafe379b221 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -118,12 +118,16 @@ if(NOT DEFINED CMAKE_C_STANDARD) set(CMAKE_C_STANDARD 11) endif() -# This ensures that things like c++11 get passed correctly +# This ensures that things like c++11/c++14 get passed correctly if(NOT DEFINED CMAKE_CXX_STANDARD) - set(CMAKE_CXX_STANDARD 11) + if(ARROW_AZURE) + set(CMAKE_CXX_STANDARD 14) + else() + set(CMAKE_CXX_STANDARD 11) + endif() endif() -# We require a C++11 compliant compiler +# We require a C++11/14 compliant compiler set(CMAKE_CXX_STANDARD_REQUIRED ON) # ARROW-6848: Do not use GNU (or other CXX) extensions diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index da2ab430ac76d..c2879fdbc9de1 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4674,20 +4674,13 @@ macro(build_azuresdk) BUILD_BYPRODUCTS ${AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY}) add_dependencies(Azure::azure-storage-files-datalake azure_storage_files_datalake_ep) - set_property(TARGET Azure::azure-core - APPEND - PROPERTY INTERFACE_LINK_LIBRARIES CURL::libcurl LibXml2::LibXml2) - set(AZURESDK_LINK_LIBRARIES ${AZURESDK_LIBRARIES}) endmacro() if(ARROW_AZURE) - # TODO - use resolve_dependency build_azuresdk() - foreach(AZURESDK_LIBRARY_CPP ${AZURESDK_LIBRARIES_CPP}) - find_package(${AZURESDK_LIBRARY_CPP} CONFIG REQUIRED) - endforeach() - include_directories(SYSTEM ${AZURESDK_INCLUDE_DIR}) + find_curl() + find_package(LibXml2 REQUIRED) message(STATUS "Found Azure SDK headers: ${AZURESDK_INCLUDE_DIR}") message(STATUS "Found Azure SDK libraries: ${AZURESDK_LINK_LIBRARIES}") endif() diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index ec6cada1cda98..d2d520bd4ddd9 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -469,6 +469,12 @@ if(ARROW_FILESYSTEM) filesystem/path_util.cc filesystem/util_internal.cc) + if(ARROW_AZURE) + list(APPEND ARROW_SRCS filesystem/azurefs.cc filesystem/azurefs_mock.cc) + set_source_files_properties(filesystem/azurefs.cc filesystem/azurefs_mock.cc + PROPERTIES SKIP_PRECOMPILE_HEADERS ON + SKIP_UNITY_BUILD_INCLUSION ON) + endif() if(ARROW_GCS) list(APPEND ARROW_SRCS filesystem/gcsfs.cc filesystem/gcsfs_internal.cc) set_source_files_properties(filesystem/gcsfs.cc filesystem/gcsfs_internal.cc @@ -573,6 +579,7 @@ add_arrow_lib(arrow SHARED_INSTALL_INTERFACE_LIBS ${ARROW_SHARED_INSTALL_INTERFACE_LIBS}) +target_link_libraries(arrow_shared PUBLIC LibXml2::LibXml2) add_dependencies(arrow ${ARROW_LIBRARIES}) if(ARROW_BUILD_STATIC AND WIN32) diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index 819eca08cf1cf..8255c01b514d0 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -38,31 +38,10 @@ if(ARROW_GCS) endif() if(ARROW_AZURE) - set(AZURE_SRCS) - list(APPEND - AZURE_SRCS - azurefs_mock.cc - azurefs.cc) - - add_arrow_lib(azurefs - SOURCES - ${AZURE_SRCS} - SHARED_LINK_LIBS - ${ARROW_LINK_LIBS} - SHARED_PRIVATE_LINK_LIBS - ${ARROW_SHARED_PRIVATE_LINK_LIBS} - STATIC_LINK_LIBS - ${ARROW_STATIC_LINK_LIBS}) - - set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) - - set(TEST_LIBS_AZURE ${ARROW_TEST_LINK_LIBS}) - list(APPEND TEST_LIBS_AZURE azurefs_shared) - add_arrow_test(azurefs_test EXTRA_LABELS filesystem - STATIC_LINK_LIBS - ${TEST_LIBS_AZURE} - ) - set_target_properties(arrow-azurefs-test PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) + add_arrow_test(azurefs_test + EXTRA_LABELS + filesystem + EXTRA_LINK_LIBS) endif() if(ARROW_S3) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index bf348a2e7942d..586b609f3644e 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -1604,25 +1604,37 @@ Status AzureBlobFileSystem::DeleteDir(const std::string& s) { return impl_->DeleteDir(path.container, path.path_to_file_parts); } -Status AzureBlobFileSystem::DeleteDirContents(const std::string& s) { +Status AzureBlobFileSystem::DeleteDirContents(const std::string& s, bool missing_dir_ok) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty()) { + if (missing_dir_ok) { + return Status::OK(); + } return Status::IOError("Invalid path provided"); } if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container).ValueOrDie())) { - return Status::IOError("Invalid path provided1"); + if (missing_dir_ok) { + return Status::OK(); + } + return Status::IOError("Invalid path provided"); } if (impl_->FileExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie()) { - return Status::IOError("Invalid path provided2"); + if (missing_dir_ok) { + return Status::OK(); + } + return Status::IOError("Invalid path provided"); } if (!(path.path_to_file.empty()) && !(impl_->DirExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { - return Status::IOError("Invalid path provided3"); + if (missing_dir_ok) { + return Status::OK(); + } + return Status::IOError("Invalid path provided"); } return impl_->DeleteDirContents(path.container, path.path_to_file, diff --git a/cpp/src/arrow/filesystem/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h index 9c4d8bbeb7f64..49d46d66c0b3a 100644 --- a/cpp/src/arrow/filesystem/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -119,7 +119,7 @@ class ARROW_EXPORT AzureBlobFileSystem : public FileSystem { Status CreateDir(const std::string& path, bool recursive = true) override; Status DeleteDir(const std::string& path) override; - Status DeleteDirContents(const std::string& path) override; + Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override; Status DeleteRootDirContents() override; Status DeleteFile(const std::string& path) override; diff --git a/cpp/src/arrow/filesystem/azurefs_mock.cc b/cpp/src/arrow/filesystem/azurefs_mock.cc index 945b8e8859a0c..92d2b8b97ad0d 100644 --- a/cpp/src/arrow/filesystem/azurefs_mock.cc +++ b/cpp/src/arrow/filesystem/azurefs_mock.cc @@ -336,16 +336,6 @@ class MockFSInputStream : public io::BufferReader { } // namespace -std::ostream& operator<<(std::ostream& os, const MockDirInfo& di) { - return os << "'" << di.full_path << "' [mtime=" << di.mtime.time_since_epoch().count() - << "]"; -} - -std::ostream& operator<<(std::ostream& os, const MockFileInfo& di) { - return os << "'" << di.full_path << "' [mtime=" << di.mtime.time_since_epoch().count() - << ", size=" << di.data.length() << "]"; -} - //////////////////////////////////////////////////////////////////////////// // MockAzureFileSystem implementation @@ -787,23 +777,35 @@ Status MockAzureFileSystem::DeleteDir(const std::string& s) { return impl_->DeleteDir(path.container, path.path_to_file_parts, path.full_path); } -Status MockAzureFileSystem::DeleteDirContents(const std::string& s) { +Status MockAzureFileSystem::DeleteDirContents(const std::string& s, bool missing_dir_ok) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty()) { + if (missing_dir_ok) { + return Status::OK(); + } return Status::IOError("Invalid path provided"); } if (path.path_to_file.empty() && !(impl_->ContainerExists(path.container).ValueOrDie())) { + if (missing_dir_ok) { + return Status::OK(); + } return Status::IOError("Invalid path provided1"); } if (impl_->FileExists(path.full_path).ValueOrDie()) { + if (missing_dir_ok) { + return Status::OK(); + } return Status::IOError("Invalid path provided2"); } if (!(path.path_to_file.empty()) && !(impl_->DirExists(path.full_path).ValueOrDie())) { + if (missing_dir_ok) { + return Status::OK(); + } return Status::IOError("Invalid path provided3"); } @@ -1143,21 +1145,6 @@ Result> MockAzureFileSystem::Make( return fs; } - -FileInfoGenerator MockAsyncFileSystem::GetFileInfoGenerator(const FileSelector& select) { - auto maybe_infos = GetFileInfo(select); - if (maybe_infos.ok()) { - // Return the FileInfo entries one by one - const auto& infos = *maybe_infos; - std::vector chunks(infos.size()); - std::transform(infos.begin(), infos.end(), chunks.begin(), - [](const FileInfo& info) { return FileInfoVector{info}; }); - return MakeVectorGenerator(std::move(chunks)); - } else { - return MakeFailingGenerator(maybe_infos); - } -} - } // namespace internal } // namespace fs } // namespace arrow diff --git a/cpp/src/arrow/filesystem/azurefs_mock.h b/cpp/src/arrow/filesystem/azurefs_mock.h index f778ae04a70f5..5ead83da08192 100644 --- a/cpp/src/arrow/filesystem/azurefs_mock.h +++ b/cpp/src/arrow/filesystem/azurefs_mock.h @@ -53,7 +53,7 @@ class ARROW_EXPORT MockAzureFileSystem : public FileSystem { Status CreateDir(const std::string& path, bool recursive = true) override; Status DeleteDir(const std::string& path) override; - Status DeleteDirContents(const std::string& path) override; + Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override; Status DeleteRootDirContents() override; Status DeleteFile(const std::string& path) override; From f99fad5c2b94ea0711274bf10d78aa5fd64cb82a Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 1 Jun 2022 12:32:06 +0530 Subject: [PATCH 09/34] Changed LibXml2 target --- cpp/cmake_modules/ThirdpartyToolchain.cmake | 1 + cpp/src/arrow/CMakeLists.txt | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index c2879fdbc9de1..84e3aab3812f8 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4631,6 +4631,7 @@ macro(build_azuresdk) INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") set("${_AZURESDK_LIB_NAME_PREFIX}_STATIC_LIBRARY" ${_AZURESDK_STATIC_LIBRARY}) + target_link_libraries(${_AZURESDK_TARGET_NAME} INTERFACE LibXml2::LibXml2) list(APPEND AZURESDK_LIBRARIES ${_AZURESDK_TARGET_NAME}) endforeach() diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index d2d520bd4ddd9..1ded8e59d49e0 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -579,7 +579,6 @@ add_arrow_lib(arrow SHARED_INSTALL_INTERFACE_LIBS ${ARROW_SHARED_INSTALL_INTERFACE_LIBS}) -target_link_libraries(arrow_shared PUBLIC LibXml2::LibXml2) add_dependencies(arrow ${ARROW_LIBRARIES}) if(ARROW_BUILD_STATIC AND WIN32) From e2008d85b837c9b9d8e7376406587a364d7b7a13 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 2 Jun 2022 13:02:05 +0530 Subject: [PATCH 10/34] Fixing CMake styling issues --- cpp/CMakeLists.txt | 2 +- cpp/cmake_modules/DefineOptions.cmake | 3 ++- cpp/cmake_modules/ThirdpartyToolchain.cmake | 3 ++- cpp/src/arrow/filesystem/CMakeLists.txt | 5 +---- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 395f54b2e267a..89159ba00b65d 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -816,7 +816,7 @@ if(ARROW_WITH_OPENTELEMETRY) opentelemetry-cpp::otlp_http_exporter) endif() -if (ARROW_AZURE) +if(ARROW_AZURE) list(APPEND ARROW_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) list(APPEND ARROW_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) endif() diff --git a/cpp/cmake_modules/DefineOptions.cmake b/cpp/cmake_modules/DefineOptions.cmake index 117a57d534810..5bf9614a27fa1 100644 --- a/cpp/cmake_modules/DefineOptions.cmake +++ b/cpp/cmake_modules/DefineOptions.cmake @@ -215,7 +215,8 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}") #---------------------------------------------------------------------- set_option_category("Project component") - define_option(ARROW_AZURE "Build Arrow with Azure support (requires the Azure SDK for C++)" OFF) + define_option(ARROW_AZURE + "Build Arrow with Azure support (requires the Azure SDK for C++)" OFF) define_option(ARROW_BUILD_UTILITIES "Build Arrow commandline utilities" OFF) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index 84e3aab3812f8..f531570a4a09b 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -465,7 +465,8 @@ else() endif() if(DEFINED ENV{ARROW_AZURE_STORAGE_FILES_DATALAKE_URL}) - set(AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_FILES_DATALAKE_URL}") + set(AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL + "$ENV{ARROW_AZURE_STORAGE_FILES_DATALAKE_URL}") else() set_urls(AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-files-datalake_${ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_VERSION}.tar.gz" diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index 8255c01b514d0..69dee45915483 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -38,10 +38,7 @@ if(ARROW_GCS) endif() if(ARROW_AZURE) - add_arrow_test(azurefs_test - EXTRA_LABELS - filesystem - EXTRA_LINK_LIBS) + add_arrow_test(azurefs_test EXTRA_LABELS filesystem EXTRA_LINK_LIBS) endif() if(ARROW_S3) From bb49f62366149f37bd874779a7af53abd691c6e5 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Fri, 3 Jun 2022 12:06:23 +0530 Subject: [PATCH 11/34] Enabling ARROW_AZURE flag --- .github/workflows/cpp.yml | 3 +++ ci/docker/ubuntu-18.04-cpp.dockerfile | 3 ++- ci/docker/ubuntu-20.04-cpp.dockerfile | 3 ++- ci/docker/ubuntu-22.04-cpp.dockerfile | 3 ++- 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index 4a42c67aea399..aa0285ba7d6c3 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -122,6 +122,7 @@ jobs: strategy: fail-fast: false env: + ARROW_AZURE: ON ARROW_BUILD_TESTS: ON ARROW_DATASET: ON ARROW_FLIGHT: ON @@ -199,6 +200,7 @@ jobs: name: Windows 2019 generator: Visual Studio 16 2019 env: + ARROW_AZURE: ON ARROW_BOOST_USE_SHARED: OFF ARROW_BUILD_BENCHMARKS: ON ARROW_BUILD_SHARED: ON @@ -268,6 +270,7 @@ jobs: - 32 - 64 env: + ARROW_AZURE: ON ARROW_BUILD_SHARED: ON ARROW_BUILD_STATIC: OFF ARROW_BUILD_TESTS: ON diff --git a/ci/docker/ubuntu-18.04-cpp.dockerfile b/ci/docker/ubuntu-18.04-cpp.dockerfile index ed038f4c4c57d..fe7a04e96c2cb 100644 --- a/ci/docker/ubuntu-18.04-cpp.dockerfile +++ b/ci/docker/ubuntu-18.04-cpp.dockerfile @@ -98,7 +98,8 @@ RUN apt-get update -y -q && \ # - thrift is too old # - utf8proc is too old(v2.1.0) # - s3 tests would require boost-asio that is included since Boost 1.66.0 -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_AZURE: ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_FLIGHT=OFF \ diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index 6e811ea2f71f5..8c118f589d303 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -122,7 +122,8 @@ RUN /arrow/ci/scripts/install_ceph.sh # - flatbuffer is not packaged # - libgtest-dev only provide sources # - libprotobuf-dev only provide sources -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_AZURE: ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_DATASET=ON \ ARROW_FLIGHT=OFF \ diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index a7cc5ff38ad8c..5143738a86247 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -150,7 +150,8 @@ RUN /arrow/ci/scripts/install_gcs_testbench.sh default # - flatbuffer is not packaged # - libgtest-dev only provide sources # - libprotobuf-dev only provide sources -ENV ARROW_BUILD_TESTS=ON \ +ENV ARROW_AZURE: ON \ + ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_DATASET=ON \ ARROW_FLIGHT=ON \ From 323b3946ace3283b8715151af28fbe2be0207ad3 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 6 Jun 2022 12:10:51 +0530 Subject: [PATCH 12/34] Added OpenSSL dependency --- cpp/cmake_modules/ThirdpartyToolchain.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index f531570a4a09b..6dda53b2ee1d3 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4682,6 +4682,7 @@ endmacro() if(ARROW_AZURE) build_azuresdk() find_curl() + find_package(OpenSSL ${ARROW_OPENSSL_REQUIRED_VERSION} REQUIRED) find_package(LibXml2 REQUIRED) message(STATUS "Found Azure SDK headers: ${AZURESDK_INCLUDE_DIR}") message(STATUS "Found Azure SDK libraries: ${AZURESDK_LINK_LIBRARIES}") From 95cc602033cc62d03e0415606593a9b5f0299c28 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 15 Jun 2022 15:16:01 +0530 Subject: [PATCH 13/34] Disabling ARROW_AZURE in windows-mingw --- .github/workflows/cpp.yml | 3 +- cpp/cmake_modules/ThirdpartyToolchain.cmake | 130 ++++++++++++++------ 2 files changed, 92 insertions(+), 41 deletions(-) diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index aa0285ba7d6c3..341f9836f4381 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -200,7 +200,6 @@ jobs: name: Windows 2019 generator: Visual Studio 16 2019 env: - ARROW_AZURE: ON ARROW_BOOST_USE_SHARED: OFF ARROW_BUILD_BENCHMARKS: ON ARROW_BUILD_SHARED: ON @@ -270,7 +269,7 @@ jobs: - 32 - 64 env: - ARROW_AZURE: ON + ARROW_AZURE: OFF ARROW_BUILD_SHARED: ON ARROW_BUILD_STATIC: OFF ARROW_BUILD_TESTS: ON diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index 6dda53b2ee1d3..b60c759a1b13a 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4597,93 +4597,145 @@ endif() macro(build_azuresdk) message(STATUS "Building Azure C++ SDK from source") + find_curl() + find_package(OpenSSL ${ARROW_OPENSSL_REQUIRED_VERSION} REQUIRED) + set(AZURESDK_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/azuresdk_ep-install") set(AZURESDK_INCLUDE_DIR "${AZURESDK_PREFIX}/include") + set(AZURESDK_LIB_DIR "lib") + + # provide hint for Azure SDK to link with the already located openssl + get_filename_component(OPENSSL_ROOT_HINT "${OPENSSL_INCLUDE_DIR}" DIRECTORY) - set(AZURESDK_CMAKE_ARGS + set(AZURESDK_COMMON_CMAKE_ARGS ${EP_COMMON_CMAKE_ARGS} - -DBUILD_TESTING=OFF - -DCMAKE_INSTALL_LIBDIR=lib + -DBUILD_SHARED_LIBS=OFF + -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} + -DCMAKE_INSTALL_LIBDIR=${AZURESDK_LIB_DIR} + -DENABLE_TESTING=OFF + -DENABLE_UNITY_BUILD=ON "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" - -DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}) + "-DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}" + -DWARNINGS_AS_ERRORS=OFF + -DOPENSSL_ROOT_DIR=${OPENSSL_ROOT_HINT}) file(MAKE_DIRECTORY ${AZURESDK_INCLUDE_DIR}) - # Azure C++ SDK related libraries to link statically - set(_AZURESDK_LIBS - azure-core - azure-identity - azure-storage-blobs - azure-storage-common - azure-storage-files-datalake) - set(AZURESDK_LIBRARIES) - set(AZURESDK_LIBRARIES_CPP) - foreach(_AZURESDK_LIB ${_AZURESDK_LIBS}) - string(TOUPPER ${_AZURESDK_LIB} _AZURESDK_LIB_UPPER) - string(REPLACE "-" "_" _AZURESDK_LIB_NAME_PREFIX ${_AZURESDK_LIB_UPPER}) - list(APPEND AZURESDK_LIBRARIES_CPP "${_AZURESDK_LIB}-cpp") - set(_AZURESDK_TARGET_NAME Azure::${_AZURESDK_LIB}) - set(_AZURESDK_STATIC_LIBRARY - "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}${_AZURESDK_LIB}${CMAKE_STATIC_LIBRARY_SUFFIX}" - ) - add_library(${_AZURESDK_TARGET_NAME} STATIC IMPORTED) - set_target_properties(${_AZURESDK_TARGET_NAME} - PROPERTIES IMPORTED_LOCATION ${_AZURESDK_STATIC_LIBRARY} - INTERFACE_INCLUDE_DIRECTORIES - "${AZURESDK_INCLUDE_DIR}") - set("${_AZURESDK_LIB_NAME_PREFIX}_STATIC_LIBRARY" ${_AZURESDK_STATIC_LIBRARY}) - target_link_libraries(${_AZURESDK_TARGET_NAME} INTERFACE LibXml2::LibXml2) - list(APPEND AZURESDK_LIBRARIES ${_AZURESDK_TARGET_NAME}) - endforeach() - + set(AZURE_CORE_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-core${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) externalproject_add(azure_core_ep ${EP_LOG_OPTIONS} + INSTALL_DIR ${AZURESDK_PREFIX} URL ${AZURE_CORE_SOURCE_URL} URL_HASH "SHA256=${ARROW_AZURE_CORE_BUILD_SHA256_CHECKSUM}" - CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_CORE_STATIC_LIBRARY}) + add_library(Azure::azure-core STATIC IMPORTED) + set_target_properties(Azure::azure-core + PROPERTIES IMPORTED_LOCATION "${AZURE_CORE_STATIC_LIBRARY}" + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + target_link_libraries(Azure::azure-core INTERFACE LibXml2::LibXml2) add_dependencies(Azure::azure-core azure_core_ep) + set(AZURE_IDENTITY_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-identity${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) externalproject_add(azure_identity_ep ${EP_LOG_OPTIONS} + INSTALL_DIR ${AZURESDK_PREFIX} URL ${AZURE_IDENTITY_SOURCE_URL} URL_HASH "SHA256=${ARROW_AZURE_IDENTITY_BUILD_SHA256_CHECKSUM}" - CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_IDENTITY_STATIC_LIBRARY}) + add_library(Azure::azure-identity STATIC IMPORTED) + set_target_properties(Azure::azure-identity + PROPERTIES IMPORTED_LOCATION "${AZURE_IDENTITY_STATIC_LIBRARY}" + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + target_link_libraries(Azure::azure-identity INTERFACE LibXml2::LibXml2) add_dependencies(Azure::azure-identity azure_identity_ep) + set(AZURE_STORAGE_BLOBS_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-storage-blobs${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) externalproject_add(azure_storage_blobs_ep ${EP_LOG_OPTIONS} + INSTALL_DIR ${AZURESDK_PREFIX} URL ${AZURE_STORAGE_BLOB_SOURCE_URL} URL_HASH "SHA256=${ARROW_AZURE_STORAGE_BLOB_BUILD_SHA256_CHECKSUM}" - CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_STORAGE_BLOBS_STATIC_LIBRARY}) + add_library(Azure::azure-storage-blobs STATIC IMPORTED) + set_target_properties(Azure::azure-storage-blobs + PROPERTIES IMPORTED_LOCATION + "${AZURE_STORAGE_BLOBS_STATIC_LIBRARY}" + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + target_link_libraries(Azure::azure-storage-blobs INTERFACE LibXml2::LibXml2) add_dependencies(Azure::azure-storage-blobs azure_storage_blobs_ep) + set(AZURE_STORAGE_COMMON_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-storage-common${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) externalproject_add(azure_storage_common_ep ${EP_LOG_OPTIONS} + INSTALL_DIR ${AZURESDK_PREFIX} URL ${AZURE_STORAGE_COMMON_SOURCE_URL} URL_HASH "SHA256=${ARROW_AZURE_STORAGE_COMMON_BUILD_SHA256_CHECKSUM}" - CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_STORAGE_COMMON_STATIC_LIBRARY}) + add_library(Azure::azure-storage-common STATIC IMPORTED) + set_target_properties(Azure::azure-storage-common + PROPERTIES IMPORTED_LOCATION + "${AZURE_STORAGE_COMMON_STATIC_LIBRARY}" + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + target_link_libraries(Azure::azure-storage-common INTERFACE LibXml2::LibXml2) add_dependencies(Azure::azure-storage-common azure_storage_common_ep) + set(AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY + "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-storage-files-datalake${CMAKE_STATIC_LIBRARY_SUFFIX}" + ) externalproject_add(azure_storage_files_datalake_ep ${EP_LOG_OPTIONS} + INSTALL_DIR ${AZURESDK_PREFIX} URL ${AZURE_STORAGE_FILES_DATALAKE_SOURCE_URL} URL_HASH "SHA256=${ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_SHA256_CHECKSUM}" - CMAKE_ARGS ${AZURESDK_CMAKE_ARGS} + CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY}) + add_library(Azure::azure-storage-files-datalake STATIC IMPORTED) + set_target_properties(Azure::azure-storage-files-datalake + PROPERTIES IMPORTED_LOCATION + "${AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY}" + INTERFACE_INCLUDE_DIRECTORIES + "${AZURESDK_INCLUDE_DIR}") + target_link_libraries(Azure::azure-storage-files-datalake INTERFACE LibXml2::LibXml2) add_dependencies(Azure::azure-storage-files-datalake azure_storage_files_datalake_ep) + set(AZURESDK_LIBRARIES) + list(APPEND + AZURESDK_LIBRARIES + Azure::azure-core + Azure::azure-storage-blobs + Azure::azure-identity + Azure::azure-storage-common + Azure::azure-storage-files-datalake) + list(APPEND + ARROW_BUNDLED_STATIC_LIBS + Azure::azure-core + Azure::azure-storage-blobs + Azure::azure-identity + Azure::azure-storage-common + Azure::azure-storage-files-datalake) + set(AZURESDK_LINK_LIBRARIES ${AZURESDK_LIBRARIES}) endmacro() if(ARROW_AZURE) - build_azuresdk() - find_curl() - find_package(OpenSSL ${ARROW_OPENSSL_REQUIRED_VERSION} REQUIRED) find_package(LibXml2 REQUIRED) + build_azuresdk() message(STATUS "Found Azure SDK headers: ${AZURESDK_INCLUDE_DIR}") message(STATUS "Found Azure SDK libraries: ${AZURESDK_LINK_LIBRARIES}") endif() From 9350b4cd968752cc32c7f5e6488300ba287d386c Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 15 Jun 2022 16:37:16 +0530 Subject: [PATCH 14/34] Fixing lint issues --- cpp/src/arrow/filesystem/azurefs.cc | 90 +++++++++++++----------- cpp/src/arrow/filesystem/azurefs_mock.cc | 6 +- 2 files changed, 51 insertions(+), 45 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 586b609f3644e..5cc8277ebea3b 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -752,9 +752,9 @@ void FileObjectToInfo( FileInfo* info) { info->set_type(FileType::File); info->set_size(static_cast(properties.FileSize)); - info->set_mtime(ToTimePoint(std::chrono::duration_cast( - properties.LastModified - Azure::DateTime(1970)) - .count())); + info->set_mtime(ToTimePoint((int)(std::chrono::duration_cast( + properties.LastModified - Azure::DateTime(1970)) + .count()))); } void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, @@ -763,8 +763,8 @@ void PathInfoToFileInfo(const std::string path, const FileType type, const int64 info->set_size(size); info->set_path(path); info->set_mtime(ToTimePoint( - std::chrono::duration_cast(dt - Azure::DateTime(1970)) - .count())); + (int)(std::chrono::duration_cast(dt - Azure::DateTime(1970)) + .count()))); } } // namespace @@ -788,8 +788,8 @@ class AzureBlobFileSystem::Impl Status Init() { dfs_endpoint_url = options_.account_dfs_url; blob_endpoint_url = options_.account_blob_url; - InitServiceClient(gen1Client_, options_, blob_endpoint_url); - InitServiceClient(gen2Client_, options_, dfs_endpoint_url); + RETURN_NOT_OK(InitServiceClient(gen1Client_, options_, blob_endpoint_url)); + RETURN_NOT_OK(InitServiceClient(gen2Client_, options_, dfs_endpoint_url)); isHierarchicalNamespaceEnabled = gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; return Status::OK(); @@ -819,8 +819,8 @@ class AzureBlobFileSystem::Impl std::string uri = s; ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file); + RETURN_NOT_OK(InitPathClient( + pathClient_, options_, uri, path.container, path.path_to_file)); try { auto properties = pathClient_->GetProperties(); return properties.Value.IsDirectory; @@ -833,8 +833,8 @@ class AzureBlobFileSystem::Impl std::string uri = s; ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file); + RETURN_NOT_OK(InitPathClient( + pathClient_, options_, uri, path.container, path.path_to_file)); try { auto properties = pathClient_->GetProperties(); return !properties.Value.IsDirectory; @@ -1085,9 +1085,10 @@ class AzureBlobFileSystem::Impl for (auto p : paths.Blobs) { std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name); + RETURN_NOT_OK( + InitPathClient( + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name)); childrenFiles->push_back(container + "/" + p.Name); } } catch (std::exception const& e) { @@ -1103,9 +1104,10 @@ class AzureBlobFileSystem::Impl for (auto p : paths.Paths) { std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name); + RETURN_NOT_OK( + InitPathClient( + pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, + container, p.Name)); if (pathClient_->GetProperties().Value.IsDirectory) { childrenDirs->push_back(container + "/" + p.Name); } else { @@ -1129,9 +1131,9 @@ class AzureBlobFileSystem::Impl auto paths = dirClient.ListPaths(false); for (auto p : paths.Paths) { std::shared_ptr pathClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, - p.Name); + p.Name)); if (pathClient_->GetProperties().Value.IsDirectory) { childrenDirs->push_back(container + "/" + p.Name); } else { @@ -1161,7 +1163,7 @@ class AzureBlobFileSystem::Impl FileInfo info; // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - GetProperties(dfs_endpoint_url + childFile, &properties); + RETURN_NOT_OK(GetProperties(dfs_endpoint_url + childFile, &properties)); PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, properties.LastModified, &info); out->push_back(std::move(info)); @@ -1170,7 +1172,7 @@ class AzureBlobFileSystem::Impl FileInfo info; // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - GetProperties(dfs_endpoint_url + childDir, &properties); + RETURN_NOT_OK(GetProperties(dfs_endpoint_url + childDir, &properties)); PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, &info); out->push_back(std::move(info)); @@ -1189,8 +1191,8 @@ class AzureBlobFileSystem::Impl Azure::Storage::Files::DataLake::Models::PathProperties* properties) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, s, path.container, path.path_to_file); + RETURN_NOT_OK(InitPathClient( + pathClient_, options_, s, path.container, path.path_to_file)); if (path.path_to_file.empty()) { auto fileSystemClient = gen2Client_->GetFileSystemClient(path.container); auto props = fileSystemClient.GetProperties().Value; @@ -1219,11 +1221,11 @@ class AzureBlobFileSystem::Impl } for (const auto& childFile : childrenFiles) { ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); - DeleteFile(filePath.container, filePath.path_to_file_parts); + RETURN_NOT_OK(DeleteFile(filePath.container, filePath.path_to_file_parts)); } for (const auto& childDir : childrenDirs) { ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); - DeleteDir(dirPath.container, dirPath.path_to_file_parts); + RETURN_NOT_OK(DeleteDir(dirPath.container, dirPath.path_to_file_parts)); } return Status::OK(); } @@ -1255,14 +1257,14 @@ class AzureBlobFileSystem::Impl return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( pathClient_, options_, dfs_endpoint_url + path.full_path, path.container, - path.path_to_file); + path.path_to_file)); std::shared_ptr fileClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( fileClient_, options_, dfs_endpoint_url + path.full_path, path.container, - path.path_to_file); + path.path_to_file)); auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path); @@ -1291,14 +1293,14 @@ class AzureBlobFileSystem::Impl return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( pathClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file); + path.path_to_file)); std::shared_ptr fileClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( fileClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file); + path.path_to_file)); if (path.has_parent()) { AzurePath parent_path = path.parent(); @@ -1335,12 +1337,12 @@ class AzureBlobFileSystem::Impl } std::shared_ptr pathClient_; - InitPathClient( - pathClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file); + RETURN_NOT_OK(InitPathClient( + pathClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file)); std::shared_ptr fileClient_; - InitPathClient( - fileClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file); + RETURN_NOT_OK(InitPathClient( + fileClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file)); auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, metadata); @@ -1370,14 +1372,14 @@ class AzureBlobFileSystem::Impl return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( pathClient_, options_, dfs_endpoint_url + info.path(), path.container, - path.path_to_file); + path.path_to_file)); std::shared_ptr fileClient_; - InitPathClient( + RETURN_NOT_OK(InitPathClient( fileClient_, options_, dfs_endpoint_url + info.path(), path.container, - path.path_to_file); + path.path_to_file)); auto ptr = std::make_shared(pathClient_, fileClient_, fs->io_context(), path, info.size()); @@ -1450,7 +1452,8 @@ Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { if (file_exists) { // "File" object found Azure::Storage::Files::DataLake::Models::PathProperties properties; - impl_->GetProperties(impl_->dfs_endpoint_url + path.full_path, &properties); + RETURN_NOT_OK( + impl_->GetProperties(impl_->dfs_endpoint_url + path.full_path, &properties)); FileObjectToInfo(properties, &info); return info; } @@ -1477,7 +1480,8 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele FileInfo info; // std::string url = impl_->gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties); + RETURN_NOT_OK( + impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties)); PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, &info); results.push_back(std::move(info)); diff --git a/cpp/src/arrow/filesystem/azurefs_mock.cc b/cpp/src/arrow/filesystem/azurefs_mock.cc index 92d2b8b97ad0d..5fd537cfa09ca 100644 --- a/cpp/src/arrow/filesystem/azurefs_mock.cc +++ b/cpp/src/arrow/filesystem/azurefs_mock.cc @@ -687,11 +687,13 @@ class MockAzureFileSystem::Impl { } for (const auto& childFile : childrenFiles) { ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); - DeleteFile(filePath.container, filePath.path_to_file_parts, filePath.full_path); + RETURN_NOT_OK(DeleteFile(filePath.container, filePath.path_to_file_parts, + filePath.full_path)); } for (const auto& childDir : childrenDirs) { ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); - DeleteDir(dirPath.container, dirPath.path_to_file_parts, dirPath.full_path); + RETURN_NOT_OK( + DeleteDir(dirPath.container, dirPath.path_to_file_parts, dirPath.full_path)); } return Status::OK(); } From 9cd1a1ada9ee4db86d4b8c5346255d68c4884792 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 15 Jun 2022 19:28:19 +0530 Subject: [PATCH 15/34] Fixing azurefs_test --- cpp/src/arrow/filesystem/azurefs_test.cc | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 1900fabfb4f21..1e683801230ff 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -46,12 +46,12 @@ class TestAzureFileSystem : public ::testing::Test { void SetUp() override { time_ = TimePoint(TimePoint::duration(42)); MakeFileSystem(); - fs_->CreateDir("container"); - fs_->CreateDir("empty-container"); - fs_->CreateDir("container2/newdir"); - fs_->CreateDir("container/emptydir"); - fs_->CreateDir("container/somedir"); - fs_->CreateDir("container/somedir/subdir"); + ASSERT_OK(fs_->CreateDir("container")); + ASSERT_OK(fs_->CreateDir("empty-container")); + ASSERT_OK(fs_->CreateDir("container2/newdir")); + ASSERT_OK(fs_->CreateDir("container/emptydir")); + ASSERT_OK(fs_->CreateDir("container/somedir")); + ASSERT_OK(fs_->CreateDir("container/somedir/subdir")); CreateFile(fs_.get(), "container/somedir/subdir/subfile", "sub data"); CreateFile(fs_.get(), "container/somefile", "some data"); } @@ -103,7 +103,7 @@ TEST_F(TestAzureFileSystem, CreateDir) { AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); auto res = fs_->OpenOutputStream("container/somedir/base.txt"); - res->get()->Write("Changed the data"); + ASSERT_OK(res->get()->Write("Changed the data")); // C/D/F AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); @@ -301,7 +301,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { TEST_F(TestAzureFileSystem, Move) { ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); - fs_->CreateDir("container/newdir/newsub/newsubsub", true); + ASSERT_OK(fs_->CreateDir("container/newdir/newsub/newsubsub", true)); ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); @@ -352,7 +352,7 @@ TEST_F(TestAzureFileSystem, CopyFile) { ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile")); - fs_->DeleteFile("container/somefile3"); + ASSERT_OK(fs_->DeleteFile("container/somefile3")); ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); } @@ -512,7 +512,7 @@ TEST_F(TestAzureFileSystem, DeleteDirContents) { // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - fs_->CreateDir("container2/newdir/subdir", true); + ASSERT_OK(fs_->CreateDir("container2/newdir/subdir", true)); // C/D ASSERT_OK(fs_->DeleteDirContents("container2/newdir")); From 8ba75aeb0d200d99089c1f4e0d9b1624a0b6df6b Mon Sep 17 00:00:00 2001 From: shefali singh Date: Sat, 25 Jun 2022 21:17:41 +0530 Subject: [PATCH 16/34] Added Azurite --- .github/workflows/cpp.yml | 5 +- ci/docker/ubuntu-18.04-cpp.dockerfile | 2 +- ci/docker/ubuntu-20.04-cpp.dockerfile | 5 +- ci/docker/ubuntu-22.04-cpp.dockerfile | 5 +- ci/scripts/install_azurite.sh | 31 + cpp/CMakeLists.txt | 6 +- cpp/cmake_modules/SetupCxxFlags.cmake | 10 +- cpp/cmake_modules/ThirdpartyToolchain.cmake | 29 +- cpp/src/arrow/CMakeLists.txt | 8 +- cpp/src/arrow/filesystem/CMakeLists.txt | 4 +- cpp/src/arrow/filesystem/azurefs.cc | 47 +- cpp/src/arrow/filesystem/azurefs.h | 18 +- cpp/src/arrow/filesystem/azurefs_mock.cc | 1152 ------------------- cpp/src/arrow/filesystem/azurefs_mock.h | 95 -- cpp/src/arrow/filesystem/azurefs_test.cc | 378 +++--- 15 files changed, 263 insertions(+), 1532 deletions(-) create mode 100644 ci/scripts/install_azurite.sh delete mode 100644 cpp/src/arrow/filesystem/azurefs_mock.cc delete mode 100644 cpp/src/arrow/filesystem/azurefs_mock.h diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index 341f9836f4381..2d6f720a3bb2b 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -161,6 +161,9 @@ jobs: - name: Install Google Cloud Storage Testbench shell: bash run: ci/scripts/install_gcs_testbench.sh default + - name: Install Azurite Storage Emulator + shell: bash + run: ci/scripts/install_azurite.sh $(pwd) - name: Setup ccache run: | ci/scripts/ccache_setup.sh @@ -269,7 +272,7 @@ jobs: - 32 - 64 env: - ARROW_AZURE: OFF + ARROW_AZURE: ON ARROW_BUILD_SHARED: ON ARROW_BUILD_STATIC: OFF ARROW_BUILD_TESTS: ON diff --git a/ci/docker/ubuntu-18.04-cpp.dockerfile b/ci/docker/ubuntu-18.04-cpp.dockerfile index fe7a04e96c2cb..8d4fa88ed69f8 100644 --- a/ci/docker/ubuntu-18.04-cpp.dockerfile +++ b/ci/docker/ubuntu-18.04-cpp.dockerfile @@ -98,7 +98,7 @@ RUN apt-get update -y -q && \ # - thrift is too old # - utf8proc is too old(v2.1.0) # - s3 tests would require boost-asio that is included since Boost 1.66.0 -ENV ARROW_AZURE: ON \ +ENV ARROW_AZURE=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DATASET=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index 8c118f589d303..2daa2c3222a2d 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -112,6 +112,9 @@ RUN /arrow/ci/scripts/install_minio.sh latest /usr/local COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default +COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ +RUN /arrow/ci/scripts/install_azurite.sh /usr/local + COPY ci/scripts/install_ceph.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_ceph.sh @@ -122,7 +125,7 @@ RUN /arrow/ci/scripts/install_ceph.sh # - flatbuffer is not packaged # - libgtest-dev only provide sources # - libprotobuf-dev only provide sources -ENV ARROW_AZURE: ON \ +ENV ARROW_AZURE=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_DATASET=ON \ diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index 5143738a86247..e105a8f47006f 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -143,6 +143,9 @@ RUN /arrow/ci/scripts/install_minio.sh latest /usr/local COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default +COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ +RUN /arrow/ci/scripts/install_azurite.sh /usr/local + # Prioritize system packages and local installation # The following dependencies will be downloaded due to missing/invalid packages # provided by the distribution: @@ -150,7 +153,7 @@ RUN /arrow/ci/scripts/install_gcs_testbench.sh default # - flatbuffer is not packaged # - libgtest-dev only provide sources # - libprotobuf-dev only provide sources -ENV ARROW_AZURE: ON \ +ENV ARROW_AZURE=ON \ ARROW_BUILD_TESTS=ON \ ARROW_DEPENDENCY_SOURCE=SYSTEM \ ARROW_DATASET=ON \ diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh new file mode 100644 index 0000000000000..9af468603bc0e --- /dev/null +++ b/ci/scripts/install_azurite.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# +# 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. + +set -e + +sudo apt-get -y install nodejs +echo "node version = `node --version`" +sudo npm install -g azurite +AZURITE_DIR=${0}/azurite +mkdir $AZURITE_DIR +which azurite +echo "azurite version = `azurite --version`" + +# Start azurite +azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 89159ba00b65d..969355b89695b 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -cmake_minimum_required(VERSION 3.5) +cmake_minimum_required(VERSION 3.12) message(STATUS "Building using CMake version: ${CMAKE_VERSION}") # Compiler id for Apple Clang is now AppleClang. @@ -714,6 +714,7 @@ endif() set(ARROW_LINK_LIBS arrow::flatbuffers arrow::hadoop) set(ARROW_STATIC_LINK_LIBS arrow::flatbuffers arrow::hadoop) set(ARROW_STATIC_INSTALL_INTERFACE_LIBS) +set(ARROW_AZURE_LINK_LIBS) if(ARROW_USE_BOOST) list(APPEND ARROW_LINK_LIBS Boost::headers) @@ -817,8 +818,7 @@ if(ARROW_WITH_OPENTELEMETRY) endif() if(ARROW_AZURE) - list(APPEND ARROW_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) - list(APPEND ARROW_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_AZURE_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) endif() if(ARROW_WITH_UTF8PROC) diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index cdaafe379b221..d3a2a1a2d256a 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -118,16 +118,12 @@ if(NOT DEFINED CMAKE_C_STANDARD) set(CMAKE_C_STANDARD 11) endif() -# This ensures that things like c++11/c++14 get passed correctly +# This ensures that things like c++11 get passed correctly if(NOT DEFINED CMAKE_CXX_STANDARD) - if(ARROW_AZURE) - set(CMAKE_CXX_STANDARD 14) - else() - set(CMAKE_CXX_STANDARD 11) - endif() + set(CMAKE_CXX_STANDARD 11) endif() -# We require a C++11/14 compliant compiler +# We require a C++11 compliant compiler set(CMAKE_CXX_STANDARD_REQUIRED ON) # ARROW-6848: Do not use GNU (or other CXX) extensions diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index b60c759a1b13a..5f4098050d141 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -4598,6 +4598,7 @@ macro(build_azuresdk) message(STATUS "Building Azure C++ SDK from source") find_curl() + find_package(LibXml2 REQUIRED) find_package(OpenSSL ${ARROW_OPENSSL_REQUIRED_VERSION} REQUIRED) set(AZURESDK_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/azuresdk_ep-install") @@ -4610,14 +4611,13 @@ macro(build_azuresdk) set(AZURESDK_COMMON_CMAKE_ARGS ${EP_COMMON_CMAKE_ARGS} -DBUILD_SHARED_LIBS=OFF - -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} -DCMAKE_INSTALL_LIBDIR=${AZURESDK_LIB_DIR} -DENABLE_TESTING=OFF -DENABLE_UNITY_BUILD=ON - "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" - "-DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}" + -DOPENSSL_ROOT_DIR=${OPENSSL_ROOT_HINT} -DWARNINGS_AS_ERRORS=OFF - -DOPENSSL_ROOT_DIR=${OPENSSL_ROOT_HINT}) + "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" + "-DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}") file(MAKE_DIRECTORY ${AZURESDK_INCLUDE_DIR}) @@ -4636,7 +4636,7 @@ macro(build_azuresdk) PROPERTIES IMPORTED_LOCATION "${AZURE_CORE_STATIC_LIBRARY}" INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") - target_link_libraries(Azure::azure-core INTERFACE LibXml2::LibXml2) + target_link_libraries(Azure::azure-core INTERFACE CURL::libcurl LibXml2::LibXml2) add_dependencies(Azure::azure-core azure_core_ep) set(AZURE_IDENTITY_STATIC_LIBRARY @@ -4654,7 +4654,7 @@ macro(build_azuresdk) PROPERTIES IMPORTED_LOCATION "${AZURE_IDENTITY_STATIC_LIBRARY}" INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") - target_link_libraries(Azure::azure-identity INTERFACE LibXml2::LibXml2) + target_link_libraries(Azure::azure-identity INTERFACE CURL::libcurl LibXml2::LibXml2) add_dependencies(Azure::azure-identity azure_identity_ep) set(AZURE_STORAGE_BLOBS_STATIC_LIBRARY @@ -4673,7 +4673,8 @@ macro(build_azuresdk) "${AZURE_STORAGE_BLOBS_STATIC_LIBRARY}" INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") - target_link_libraries(Azure::azure-storage-blobs INTERFACE LibXml2::LibXml2) + target_link_libraries(Azure::azure-storage-blobs + INTERFACE Azure::azure-core CURL::libcurl LibXml2::LibXml2) add_dependencies(Azure::azure-storage-blobs azure_storage_blobs_ep) set(AZURE_STORAGE_COMMON_STATIC_LIBRARY @@ -4692,8 +4693,11 @@ macro(build_azuresdk) "${AZURE_STORAGE_COMMON_STATIC_LIBRARY}" INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") - target_link_libraries(Azure::azure-storage-common INTERFACE LibXml2::LibXml2) + target_link_libraries(Azure::azure-storage-common INTERFACE CURL::libcurl + LibXml2::LibXml2) add_dependencies(Azure::azure-storage-common azure_storage_common_ep) + set_property(TARGET Azure::azure-storage-common PROPERTY INTERFACE_LINK_LIBRARIES + OpenSSL::Crypto) set(AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY "${AZURESDK_PREFIX}/lib/${CMAKE_STATIC_LIBRARY_PREFIX}azure-storage-files-datalake${CMAKE_STATIC_LIBRARY_SUFFIX}" @@ -4711,7 +4715,13 @@ macro(build_azuresdk) "${AZURE_STORAGE_FILES_DATALAKE_STATIC_LIBRARY}" INTERFACE_INCLUDE_DIRECTORIES "${AZURESDK_INCLUDE_DIR}") - target_link_libraries(Azure::azure-storage-files-datalake INTERFACE LibXml2::LibXml2) + target_link_libraries(Azure::azure-storage-files-datalake + INTERFACE Azure::azure-core + Azure::azure-identity + Azure::azure-storage-blobs + Azure::azure-storage-common + CURL::libcurl + LibXml2::LibXml2) add_dependencies(Azure::azure-storage-files-datalake azure_storage_files_datalake_ep) set(AZURESDK_LIBRARIES) @@ -4734,7 +4744,6 @@ macro(build_azuresdk) endmacro() if(ARROW_AZURE) - find_package(LibXml2 REQUIRED) build_azuresdk() message(STATUS "Found Azure SDK headers: ${AZURESDK_INCLUDE_DIR}") message(STATUS "Found Azure SDK libraries: ${AZURESDK_LINK_LIBRARIES}") diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 1ded8e59d49e0..4041a399cb193 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -470,7 +470,7 @@ if(ARROW_FILESYSTEM) filesystem/util_internal.cc) if(ARROW_AZURE) - list(APPEND ARROW_SRCS filesystem/azurefs.cc filesystem/azurefs_mock.cc) + set(AZURE_SRCS filesystem/azurefs.cc) set_source_files_properties(filesystem/azurefs.cc filesystem/azurefs_mock.cc PROPERTIES SKIP_PRECOMPILE_HEADERS ON SKIP_UNITY_BUILD_INCLUSION ON) @@ -554,6 +554,12 @@ if(${CMAKE_SYSTEM_NAME} STREQUAL "Linux" AND ${CMAKE_SYSTEM_PROCESSOR} MATCHES " string(APPEND ARROW_PC_LIBS_PRIVATE " -latomic") endif() +add_library(azurefs_objlib OBJECT ${AZURE_SRCS}) +target_link_libraries(azurefs_objlib PUBLIC ${ARROW_AZURE_LINK_LIBS}) + +set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) +list(APPEND ARROW_LINK_LIBS ${ARROW_AZURE_LINK_LIBS} azurefs_objlib) + add_arrow_lib(arrow CMAKE_PACKAGE_NAME Arrow diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index 69dee45915483..9a42ecc89f327 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -38,7 +38,9 @@ if(ARROW_GCS) endif() if(ARROW_AZURE) - add_arrow_test(azurefs_test EXTRA_LABELS filesystem EXTRA_LINK_LIBS) + add_arrow_test(azurefs_test EXTRA_LABELS filesystem) + set_target_properties(arrow-azurefs-test PROPERTIES CXX_STANDARD 14 + CXX_STANDARD_REQUIRED ON) endif() if(ARROW_S3) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 5cc8277ebea3b..e1ed328cc2d9a 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -251,11 +251,15 @@ struct AzurePath { // path_to_file = testdir/testfile.txt // path_to_file_parts = [testdir, testfile.txt] - // Expected input here => s = /synapsemlfs/testdir/testfile.txt + // Expected input here => s = synapsemlfs/testdir/testfile.txt auto src = internal::RemoveTrailingSlash(s); - if (src.starts_with("https:") || src.starts_with("http::")) { - RemoveSchemeFromUri(src); + if ((src.find("127.0.0.1") != std::string::npos)) { + RETURN_NOT_OK(FromLocalHostString(src)); } + if (internal::IsLikelyUri(src)) { + RETURN_NOT_OK(ExtractBlobPath(src)); + } + src = internal::RemoveLeadingSlash(src); auto first_sep = src.find_first_of(kSep); if (first_sep == 0) { return Status::Invalid("Path cannot start with a separator ('", s, "')"); @@ -272,9 +276,22 @@ struct AzurePath { return path; } - static void RemoveSchemeFromUri(nonstd::sv_lite::string_view& s) { - auto first = s.find(".core.windows.net"); - s = s.substr(first + 18, s.length()); + static Status FromLocalHostString(util::string_view& src) { + auto port = src.find("127.0.0.1"); + src = src.substr(port); + auto first_sep = src.find_first_of(kSep); + src = src.substr(first_sep + 1); + auto sec_sep = src.find_first_of(kSep); + src = src.substr(sec_sep + 1); + return Status::OK(); + } + + // Removes scheme, host and port from the uri + static Status ExtractBlobPath(util::string_view& s) { + Uri uri; + RETURN_NOT_OK(uri.Parse(s.to_string())); + s = uri.path(); + return Status::OK(); } static Status Validate(const AzurePath* path) { @@ -790,8 +807,14 @@ class AzureBlobFileSystem::Impl blob_endpoint_url = options_.account_blob_url; RETURN_NOT_OK(InitServiceClient(gen1Client_, options_, blob_endpoint_url)); RETURN_NOT_OK(InitServiceClient(gen2Client_, options_, dfs_endpoint_url)); - isHierarchicalNamespaceEnabled = - gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + if (options_.isTestEnabled) { + // gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled throws error + // in azurite + isHierarchicalNamespaceEnabled = false; + } else { + isHierarchicalNamespaceEnabled = + gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + } return Status::OK(); } @@ -909,7 +932,11 @@ class AzureBlobFileSystem::Impl if (!FileExists(fileClient.GetUrl()).ValueOrDie()) { return Status::IOError("Cannot delete File, Invalid File Path"); } - fileClient.DeleteIfExists(); + try { + fileClient.DeleteIfExists(); + } catch (std::exception const& e) { + // Azurite throws an exception + } return Status::OK(); } std::string file_name = path.back(); @@ -1253,7 +1280,7 @@ class AzureBlobFileSystem::Impl " hierarchical namespace not enabled"); } } - if (!(FileExists(dfs_endpoint_url + path.full_path)).ValueOrDie()) { + if (!(FileExists(dfs_endpoint_url + path.full_path).ValueOrDie())) { return Status::IOError("Invalid path provided"); } std::shared_ptr pathClient_; diff --git a/cpp/src/arrow/filesystem/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h index 49d46d66c0b3a..4b4ee7f21715f 100644 --- a/cpp/src/arrow/filesystem/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -17,8 +17,6 @@ #pragma once -#include -#include #include #include #include @@ -27,6 +25,21 @@ #include "arrow/util/macros.h" #include "arrow/util/uri.h" +namespace Azure { +namespace Core { +namespace Credentials { + +class TokenCredential; + +} // namespace Credentials +} // namespace Core +namespace Storage { + +class StorageSharedKeyCredential; + +} // namespace Storage +} // namespace Azure + namespace arrow { namespace fs { @@ -48,6 +61,7 @@ struct ARROW_EXPORT AzureOptions { std::string scheme; std::string account_dfs_url; std::string account_blob_url; + bool isTestEnabled = false; AzureCredentialsKind credentials_kind = AzureCredentialsKind::Anonymous; std::string sas_token; diff --git a/cpp/src/arrow/filesystem/azurefs_mock.cc b/cpp/src/arrow/filesystem/azurefs_mock.cc deleted file mode 100644 index 5fd537cfa09ca..0000000000000 --- a/cpp/src/arrow/filesystem/azurefs_mock.cc +++ /dev/null @@ -1,1152 +0,0 @@ -// 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_mock.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "arrow/buffer.h" -#include "arrow/buffer_builder.h" -#include "arrow/filesystem/mockfs.h" -#include "arrow/filesystem/path_util.h" -#include "arrow/filesystem/util_internal.h" -#include "arrow/io/interfaces.h" -#include "arrow/io/memory.h" -#include "arrow/util/async_generator.h" -#include "arrow/util/future.h" -#include "arrow/util/logging.h" -#include "arrow/util/string_view.h" -#include "arrow/util/variant.h" -#include "arrow/util/windows_fixup.h" - -namespace arrow { -namespace fs { -namespace internal { - -namespace { - -Status ValidatePath(util::string_view s) { - if (internal::IsLikelyUri(s)) { - return Status::Invalid("Expected a filesystem path, got a URI: '", s, "'"); - } - return Status::OK(); -} - -//////////////////////////////////////////////////////////////////////////// -// Filesystem structure - -class Entry; - -struct AzurePath { - std::string full_path; - std::string container; - std::string path_to_file; - std::vector path_to_file_parts; - - static Result FromString(const std::string& s) { - // https://synapsemladlsgen2.dfs.core.windows.net/synapsemlfs/testdir/testfile.txt - // container = synapsemlfs - // account_name = synapsemladlsgen2 - // path_to_file = testdir/testfile.txt - // path_to_file_parts = [testdir, testfile.txt] - - // Expected input here => s = /synapsemlfs/testdir/testfile.txt - auto src = internal::RemoveTrailingSlash(s); - if (src.starts_with("https:") || src.starts_with("http::")) { - RemoveSchemeFromUri(src); - } - auto first_sep = src.find_first_of(kSep); - if (first_sep == 0) { - return Status::Invalid("Path cannot start with a separator ('", s, "')"); - } - if (first_sep == std::string::npos) { - return AzurePath{std::string(src), std::string(src), "", {}}; - } - AzurePath path; - path.full_path = std::string(src); - path.container = std::string(src.substr(0, first_sep)); - path.path_to_file = std::string(src.substr(first_sep + 1)); - path.path_to_file_parts = internal::SplitAbstractPath(path.path_to_file); - RETURN_NOT_OK(Validate(&path)); - return path; - } - - static void RemoveSchemeFromUri(nonstd::sv_lite::string_view& s) { - auto first = s.find(".core.windows.net"); - s = s.substr(first + 18, s.length()); - } - - static Status Validate(const AzurePath* path) { - auto result = internal::ValidateAbstractPathParts(path->path_to_file_parts); - if (!result.ok()) { - return Status::Invalid(result.message(), " in path ", path->full_path); - } else { - return result; - } - } - - AzurePath parent() const { - DCHECK(!path_to_file_parts.empty()); - auto parent = AzurePath{"", container, "", path_to_file_parts}; - parent.path_to_file_parts.pop_back(); - parent.path_to_file = internal::JoinAbstractPath(parent.path_to_file_parts); - if (parent.path_to_file.empty()) { - parent.full_path = parent.container; - } else { - parent.full_path = parent.container + kSep + parent.path_to_file; - } - return parent; - } - - bool has_parent() const { return !path_to_file.empty(); } - - bool empty() const { return container.empty() && path_to_file.empty(); } - - bool operator==(const AzurePath& other) const { - return container == other.container && path_to_file == other.path_to_file; - } -}; - -struct File { - TimePoint mtime; - std::string name; - std::string path; - std::shared_ptr data; - std::shared_ptr metadata; - - File(TimePoint mtime, std::string name, std::string path) - : mtime(mtime), name(std::move(name)), path(std::move(path)) {} - - int64_t size() const { return data ? data->size() : 0; } - - explicit operator util::string_view() const { - if (data) { - return util::string_view(*data); - } else { - return ""; - } - } -}; - -struct Directory { - std::string name; - std::string path; - TimePoint mtime; - std::map> entries; - - Directory(std::string name, std::string path, TimePoint mtime) - : name(std::move(name)), path(std::move(path)), mtime(mtime) {} - Directory(Directory&& other) noexcept - : name(std::move(other.name)), - path(other.path), - mtime(other.mtime), - entries(std::move(other.entries)) {} - - Directory& operator=(Directory&& other) noexcept { - name = std::move(other.name); - mtime = other.mtime; - path = std::move(other.path); - entries = std::move(other.entries); - return *this; - } - - Entry* Find(const std::string& s) { - auto it = entries.find(s); - if (it != entries.end()) { - return it->second.get(); - } else { - return nullptr; - } - } - - bool CreateEntry(const std::string& s, std::unique_ptr entry) { - DCHECK(!s.empty()); - auto p = entries.emplace(s, std::move(entry)); - return p.second; - } - - void AssignEntry(const std::string& s, std::unique_ptr entry) { - DCHECK(!s.empty()); - entries[s] = std::move(entry); - } - - bool DeleteEntry(const std::string& s) { return entries.erase(s) > 0; } - - private: - ARROW_DISALLOW_COPY_AND_ASSIGN(Directory); -}; - -// A filesystem entry -using EntryBase = util::Variant; - -class Entry : public EntryBase { - public: - Entry(Entry&&) = default; - Entry& operator=(Entry&&) = default; - explicit Entry(Directory&& v) : EntryBase(std::move(v)) {} - explicit Entry(File&& v) : EntryBase(std::move(v)) {} - - bool is_dir() const { return util::holds_alternative(*this); } - - bool is_file() const { return util::holds_alternative(*this); } - - Directory& as_dir() { return util::get(*this); } - - File& as_file() { return util::get(*this); } - - // Get info for this entry. Note the path() property isn't set. - FileInfo GetInfo() { - FileInfo info; - if (is_dir()) { - Directory& dir = as_dir(); - info.set_type(FileType::Directory); - info.set_mtime(dir.mtime); - info.set_path(dir.path); - } else { - DCHECK(is_file()); - File& file = as_file(); - info.set_type(FileType::File); - info.set_mtime(file.mtime); - info.set_size(file.size()); - info.set_path(file.path); - } - return info; - } - - // Get info for this entry, knowing the parent path. - FileInfo GetInfo(const std::string& base_path) { - FileInfo info; - if (is_dir()) { - Directory& dir = as_dir(); - info.set_type(FileType::Directory); - info.set_mtime(dir.mtime); - info.set_path(ConcatAbstractPath(base_path, dir.name)); - } else { - DCHECK(is_file()); - File& file = as_file(); - info.set_type(FileType::File); - info.set_mtime(file.mtime); - info.set_size(file.size()); - info.set_path(ConcatAbstractPath(base_path, file.name)); - } - return info; - } - - // Set the entry name - void SetName(const std::string& name) { - if (is_dir()) { - as_dir().name = name; - } else { - DCHECK(is_file()); - as_file().name = name; - } - } - - private: - ARROW_DISALLOW_COPY_AND_ASSIGN(Entry); -}; - -//////////////////////////////////////////////////////////////////////////// -// Streams - -class MockFSOutputStream : public io::OutputStream { - public: - MockFSOutputStream(File* file, MemoryPool* pool) - : file_(file), builder_(pool), closed_(false) {} - - ~MockFSOutputStream() override = default; - - // Implement the OutputStream interface - Status Close() override { - if (!closed_) { - RETURN_NOT_OK(builder_.Finish(&file_->data)); - closed_ = true; - } - return Status::OK(); - } - - Status Abort() override { - if (!closed_) { - // MockFSOutputStream is mainly used for debugging and testing, so - // mark an aborted file's contents explicitly. - std::stringstream ss; - ss << "MockFSOutputStream aborted after " << file_->size() << " bytes written"; - file_->data = Buffer::FromString(ss.str()); - closed_ = true; - } - return Status::OK(); - } - - bool closed() const override { return closed_; } - - Result Tell() const override { - if (closed_) { - return Status::Invalid("Invalid operation on closed stream"); - } - return builder_.length(); - } - - Status Write(const void* data, int64_t nbytes) override { - if (closed_) { - return Status::Invalid("Invalid operation on closed stream"); - } - return builder_.Append(data, nbytes); - } - - protected: - File* file_; - BufferBuilder builder_; - bool closed_; -}; - -class MockFSInputStream : public io::BufferReader { - public: - explicit MockFSInputStream(const File& file) - : io::BufferReader(file.data), metadata_(file.metadata) {} - - Result> ReadMetadata() override { - return metadata_; - } - - protected: - std::shared_ptr metadata_; -}; - -} // namespace - -//////////////////////////////////////////////////////////////////////////// -// MockAzureFileSystem implementation - -class MockAzureFileSystem::Impl { - public: - TimePoint current_time; - MemoryPool* pool; - - // The root directory - Entry root; - std::mutex mutex; - - Impl(TimePoint current_time, MemoryPool* pool) - : current_time(current_time), pool(pool), root(Directory("", "", current_time)) {} - - std::unique_lock lock_guard() { - return std::unique_lock(mutex); - } - - Directory& RootDir() { return root.as_dir(); } - - template - Entry* FindEntry(It it, It end, size_t* nconsumed) { - size_t consumed = 0; - Entry* entry = &root; - - for (; it != end; ++it) { - const std::string& part = *it; - DCHECK(entry->is_dir()); - Entry* child = entry->as_dir().Find(part); - if (child == nullptr) { - // Partial find only - break; - } - ++consumed; - entry = child; - if (entry->is_file()) { - // Cannot go any further - break; - } - // Recurse - } - *nconsumed = consumed; - return entry; - } - - // Find an entry, allowing partial matching - Entry* FindEntry(const std::vector& parts, size_t* nconsumed) { - return FindEntry(parts.begin(), parts.end(), nconsumed); - } - - // Find an entry, only full matching allowed - Entry* FindEntry(const std::vector& parts) { - size_t consumed; - auto entry = FindEntry(parts, &consumed); - return (consumed == parts.size()) ? entry : nullptr; - } - - // Find the parent entry, only full matching allowed - Entry* FindParent(const std::vector& parts) { - if (parts.size() == 0) { - return nullptr; - } - size_t consumed; - auto entry = FindEntry(parts.begin(), --parts.end(), &consumed); - return (consumed == parts.size() - 1) ? entry : nullptr; - } - - bool CheckFile(const std::string& prefix, const Directory& dir, - const MockFileInfo& expected) { - std::string path = prefix + dir.name; - if (!path.empty()) { - path += "/"; - } - for (const auto& pair : dir.entries) { - Entry* child = pair.second.get(); - if (child->is_file()) { - auto& file = child->as_file(); - if ((path + file.name) == expected.full_path) { - if (util::string_view(file) == expected.data) { - return true; - } - } - } - } - bool res = false; - for (const auto& pair : dir.entries) { - Entry* child = pair.second.get(); - if (child->is_dir()) { - res = res || CheckFile(path, child->as_dir(), expected); - } - } - return res; - } - - Result> OpenOutputStream( - const std::string& path, bool append, - const std::shared_ptr& metadata) { - auto parts = SplitAbstractPath(path); - RETURN_NOT_OK(ValidateAbstractPathParts(parts)); - - Entry* parent = FindParent(parts); - if (parent == nullptr || !parent->is_dir()) { - return PathNotFound(path); - } - // Find the file in the parent dir, or create it - const auto& name = parts.back(); - Entry* child = parent->as_dir().Find(name); - File* file; - if (child == nullptr) { - child = new Entry(File(current_time, name, path)); - parent->as_dir().AssignEntry(name, std::unique_ptr(child)); - file = &child->as_file(); - file->path = path; - } else if (child->is_file()) { - file = &child->as_file(); - file->mtime = current_time; - file->path = path; - } else { - return NotAFile(path); - } - file->metadata = metadata; - auto ptr = std::make_shared(file, pool); - if (append && file->data) { - RETURN_NOT_OK(ptr->Write(file->data->data(), file->data->size())); - } - return ptr; - } - - Result> OpenInputReader(const std::string& path) { - auto parts = SplitAbstractPath(path); - RETURN_NOT_OK(ValidateAbstractPathParts(parts)); - - Entry* entry = FindEntry(parts); - if (entry == nullptr) { - return PathNotFound(path); - } - if (!entry->is_file()) { - return NotAFile(path); - } - return std::make_shared(entry->as_file()); - } - - // Create a container. Successful if container already exists. - Status CreateContainer(const std::string& container) { - auto parts = SplitAbstractPath(container); - size_t consumed; - Entry* entry = FindEntry(parts, &consumed); - if (consumed != 0) { - return Status::OK(); - } - std::unique_ptr child( - new Entry(Directory(container, container, current_time))); - child.get(); - bool inserted = entry->as_dir().CreateEntry(container, std::move(child)); - DCHECK(inserted); - return Status::OK(); - } - - // Tests to see if a container exists - Result ContainerExists(const std::string& container) { - auto parts = SplitAbstractPath(container); - size_t consumed; - FindEntry(parts, &consumed); - if (consumed != 0) { - return true; - } - return false; - } - - Result DirExists(const std::string& s) { - auto parts = SplitAbstractPath(s); - Entry* entry = FindEntry(parts); - if (entry == nullptr || !entry->is_dir()) { - return false; - } - return true; - } - - Result FileExists(const std::string& s) { - auto parts = SplitAbstractPath(s); - Entry* entry = FindEntry(parts); - if (entry == nullptr || !entry->is_file()) { - return false; - } - return true; - } - - Status CreateEmptyDir(const std::string& container, - const std::vector& path) { - std::vector parts = path; - parts.insert(parts.begin(), container); - size_t consumed; - Entry* entry = FindEntry(parts, &consumed); - if (!entry->is_dir()) { - auto file_path = JoinAbstractPath(parts.begin(), parts.begin() + consumed); - return Status::IOError("Cannot create directory: ", "ancestor '", file_path, - "' is not a directory"); - } - std::string str; - for (size_t i = 0; i < consumed; ++i) { - str += parts[i]; - if ((i + 1) < consumed) { - str += "/"; - } - } - for (size_t i = consumed; i < parts.size(); ++i) { - const auto& name = parts[i]; - str += "/"; - str += name; - std::unique_ptr child(new Entry(Directory(name, str, current_time))); - Entry* child_ptr = child.get(); - bool inserted = entry->as_dir().CreateEntry(name, std::move(child)); - DCHECK(inserted); - entry = child_ptr; - } - return Status::OK(); - } - - Status DeleteContainer(const std::string& container, Directory& rootDir) { - auto child = rootDir.Find(container); - if (child == nullptr) { - return Status::OK(); - } - bool deleted = rootDir.DeleteEntry(container); - DCHECK(deleted); - return Status::OK(); - } - - Status DeleteDir(const std::string& container, const std::vector& path, - const std::string& path_to_dir) { - std::vector parts = path; - parts.insert(parts.begin(), container); - - Entry* parent = FindParent(parts); - if (parent == nullptr || !parent->is_dir()) { - return PathNotFound(path_to_dir); - } - Directory& parent_dir = parent->as_dir(); - auto child = parent_dir.Find(parts.back()); - if (child == nullptr) { - return PathNotFound(path_to_dir); - } - if (!child->is_dir()) { - return NotADir(path_to_dir); - } - - bool deleted = parent_dir.DeleteEntry(parts.back()); - DCHECK(deleted); - return Status::OK(); - } - - Status DeleteFile(const std::string& container, const std::vector& path, - const std::string& path_to_file) { - if (path.empty()) { - return Status::IOError("Cannot delete File, Invalid File Path"); - } - std::vector parts = path; - parts.insert(parts.begin(), container); - - Entry* parent = FindParent(parts); - if (parent == nullptr || !parent->is_dir()) { - return PathNotFound(path_to_file); - } - Directory& parent_dir = parent->as_dir(); - auto child = parent_dir.Find(parts.back()); - if (child == nullptr) { - return PathNotFound(path_to_file); - } - if (!child->is_file()) { - return NotAFile(path_to_file); - } - bool deleted = parent_dir.DeleteEntry(parts.back()); - DCHECK(deleted); - return Status::OK(); - } - - Status ListPaths(const std::string& container, const std::string& path, - std::vector* childrenDirs, - std::vector* childrenFiles, - const bool allow_not_found = false) { - auto parts = SplitAbstractPath(path); - parts.insert(parts.begin(), container); - Entry* entry = FindEntry(parts); - Directory& base_dir = entry->as_dir(); - try { - for (const auto& pair : base_dir.entries) { - Entry* child = pair.second.get(); - if (child->is_dir()) { - childrenDirs->push_back(child->GetInfo().path()); - } - if (child->is_file()) { - childrenFiles->push_back(child->GetInfo().path()); - } - } - } catch (std::exception const& e) { - if (!allow_not_found) { - return Status::IOError("Path does not exists"); - } - } - return Status::OK(); - } - - Status Walk(const FileSelector& select, const std::string& container, - const std::string& path, int nesting_depth, std::vector* out) { - std::vector childrenDirs; - std::vector childrenFiles; - - Status st = - ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); - if (!st.ok()) { - return st; - } - - for (const auto& childFile : childrenFiles) { - FileInfo info; - auto parts = SplitAbstractPath(childFile); - Entry* entry = FindEntry(parts); - info = entry->GetInfo(); - out->push_back(std::move(info)); - } - for (const auto& childDir : childrenDirs) { - FileInfo info; - auto parts = SplitAbstractPath(childDir); - Entry* entry = FindEntry(parts); - if (entry == nullptr) { - return Status::OK(); - } - info = entry->GetInfo(); - out->push_back(std::move(info)); - if (select.recursive && nesting_depth < select.max_recursion) { - const auto src = internal::RemoveTrailingSlash(childDir); - auto first_sep = src.find_first_of("/"); - std::string s = std::string(src.substr(first_sep + 1)); - RETURN_NOT_OK(Walk(select, container, s, nesting_depth + 1, out)); - } - } - return Status::OK(); - } - - Status DeleteDirContents(const std::string& container, const std::string& path, - const std::vector& path_to_file_parts) { - std::vector childrenDirs; - std::vector childrenFiles; - - Status st = ListPaths(container, path, &childrenDirs, &childrenFiles); - if (!st.ok()) { - return st; - } - for (const auto& childFile : childrenFiles) { - ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); - RETURN_NOT_OK(DeleteFile(filePath.container, filePath.path_to_file_parts, - filePath.full_path)); - } - for (const auto& childDir : childrenDirs) { - ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); - RETURN_NOT_OK( - DeleteDir(dirPath.container, dirPath.path_to_file_parts, dirPath.full_path)); - } - return Status::OK(); - } - - Result> ListContainers(const Directory& base_dir) { - std::vector containers; - for (const auto& pair : base_dir.entries) { - Entry* child = pair.second.get(); - containers.push_back(child->GetInfo().path()); - } - return containers; - } -}; - -MockAzureFileSystem::~MockAzureFileSystem() = default; - -MockAzureFileSystem::MockAzureFileSystem(TimePoint current_time, - const io::IOContext& io_context) { - impl_ = std::unique_ptr(new Impl(current_time, io_context.pool())); -} - -bool MockAzureFileSystem::Equals(const FileSystem& other) const { return this == &other; } - -Status MockAzureFileSystem::CreateDir(const std::string& s, bool recursive) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - - if (path.empty()) { - return Status::IOError("Cannot create directory, root path given"); - } - if ((impl_->FileExists(path.full_path)).ValueOrDie()) { - return Status::IOError("Cannot create directory, file exists at path"); - } - if (path.path_to_file.empty()) { - // Create container - return impl_->CreateContainer(path.container); - } - if (recursive) { - // Ensure container exists - ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); - if (!container_exists) { - RETURN_NOT_OK(impl_->CreateContainer(path.container)); - } - std::vector parent_path_to_file; - - for (const auto& part : path.path_to_file_parts) { - parent_path_to_file.push_back(part); - RETURN_NOT_OK(impl_->CreateEmptyDir(path.container, parent_path_to_file)); - } - return Status::OK(); - } else { - // Check parent dir exists - if (path.has_parent()) { - AzurePath parent_path = path.parent(); - if (parent_path.path_to_file.empty()) { - auto exists = impl_->ContainerExists(parent_path.container); - if (!(exists.ValueOrDie())) { - return Status::IOError("Cannot create directory '", path.full_path, - "': parent directory does not exist"); - } - } else { - auto exists = impl_->DirExists(parent_path.full_path); - if (!(exists.ValueOrDie())) { - return Status::IOError("Cannot create directory '", path.full_path, - "': parent directory does not exist"); - } - } - } - return impl_->CreateEmptyDir(path.container, path.path_to_file_parts); - } -} - -Status MockAzureFileSystem::DeleteDir(const std::string& s) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - if (path.empty()) { - return Status::NotImplemented("Cannot delete all Azure Containers"); - } - if (path.path_to_file.empty()) { - return impl_->DeleteContainer(path.container, impl_->RootDir()); - } - if ((impl_->FileExists(path.full_path)).ValueOrDie()) { - return Status::IOError("Cannot delete directory, file exists at path"); - } - return impl_->DeleteDir(path.container, path.path_to_file_parts, path.full_path); -} - -Status MockAzureFileSystem::DeleteDirContents(const std::string& s, bool missing_dir_ok) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - - if (path.empty()) { - if (missing_dir_ok) { - return Status::OK(); - } - return Status::IOError("Invalid path provided"); - } - - if (path.path_to_file.empty() && - !(impl_->ContainerExists(path.container).ValueOrDie())) { - if (missing_dir_ok) { - return Status::OK(); - } - return Status::IOError("Invalid path provided1"); - } - - if (impl_->FileExists(path.full_path).ValueOrDie()) { - if (missing_dir_ok) { - return Status::OK(); - } - return Status::IOError("Invalid path provided2"); - } - - if (!(path.path_to_file.empty()) && !(impl_->DirExists(path.full_path).ValueOrDie())) { - if (missing_dir_ok) { - return Status::OK(); - } - return Status::IOError("Invalid path provided3"); - } - - return impl_->DeleteDirContents(path.container, path.path_to_file, - path.path_to_file_parts); -} - -Status MockAzureFileSystem::DeleteRootDirContents() { - auto guard = impl_->lock_guard(); - - impl_->RootDir().entries.clear(); - return Status::OK(); -} - -Status MockAzureFileSystem::DeleteFile(const std::string& s) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - return impl_->DeleteFile(path.container, path.path_to_file_parts, path.full_path); -} - -Result MockAzureFileSystem::GetFileInfo(const std::string& s) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - FileInfo info; - info.set_path(s); - - if (path.empty()) { - // It's the root path "" - info.set_type(FileType::Directory); - return info; - } else if (path.path_to_file.empty()) { - // It's a container - ARROW_ASSIGN_OR_RAISE(bool container_exists, impl_->ContainerExists(path.container)); - if (!container_exists) { - info.set_type(FileType::NotFound); - return info; - } - info.set_type(FileType::Directory); - return info; - } else { - // It's an object - ARROW_ASSIGN_OR_RAISE(bool file_exists, impl_->FileExists(path.full_path)); - if (file_exists) { - // "File" object found - std::vector parts = path.path_to_file_parts; - parts.insert(parts.begin(), path.container); - Entry* entry = impl_->FindEntry(parts); - info = entry->GetInfo(); - info.set_path(s); - return info; - } - // Not found => perhaps it's a "directory" - auto is_dir = impl_->DirExists(path.full_path); - if (is_dir.ValueOrDie()) { - info.set_type(FileType::Directory); - } else { - info.set_type(FileType::NotFound); - } - return info; - } -} - -Result MockAzureFileSystem::GetFileInfo(const FileSelector& select) { - ARROW_ASSIGN_OR_RAISE(auto base_path, AzurePath::FromString(select.base_dir)); - - FileInfoVector results; - - if (base_path.empty()) { - // List all containers - ARROW_ASSIGN_OR_RAISE(auto containers, impl_->ListContainers(impl_->RootDir())); - for (const auto& container : containers) { - FileInfo info; - auto parts = SplitAbstractPath(container); - Entry* entry = impl_->FindEntry(parts); - info = entry->GetInfo(); - info.set_path(container); - results.push_back(std::move(info)); - if (select.recursive) { - RETURN_NOT_OK(impl_->Walk(select, container, "", 0, &results)); - } - } - return results; - } - - if (base_path.path_to_file.empty() && - !(impl_->ContainerExists(base_path.container).ValueOrDie())) { - if (!select.allow_not_found) { - return Status::IOError("Invalid path provided"); - } - return results; - } - - if (impl_->FileExists(base_path.full_path).ValueOrDie()) { - return Status::IOError("Invalid path provided"); - } - - if (!(base_path.path_to_file.empty()) && - !(impl_->DirExists(base_path.full_path).ValueOrDie())) { - if (!select.allow_not_found) { - return Status::IOError("Invalid path provided"); - } - return results; - } - - // Nominal case -> walk a single container - RETURN_NOT_OK( - impl_->Walk(select, base_path.container, base_path.path_to_file, 0, &results)); - return results; -} - -namespace { - -// Helper for binary operations (move, copy) -struct BinaryOp { - std::vector src_parts; - std::vector dest_parts; - Directory& src_dir; - Directory& dest_dir; - std::string src_name; - std::string dest_name; - Entry* src_entry; - Entry* dest_entry; - - template - static Status Run(MockAzureFileSystem::Impl* impl, const std::string& src, - const std::string& dest, OpFunc&& op_func) { - RETURN_NOT_OK(ValidatePath(src)); - RETURN_NOT_OK(ValidatePath(dest)); - auto src_parts = SplitAbstractPath(src); - auto dest_parts = SplitAbstractPath(dest); - RETURN_NOT_OK(ValidateAbstractPathParts(src_parts)); - RETURN_NOT_OK(ValidateAbstractPathParts(dest_parts)); - - auto guard = impl->lock_guard(); - - // Both source and destination must have valid parents - Entry* src_parent = impl->FindParent(src_parts); - if (src_parent == nullptr || !src_parent->is_dir()) { - return PathNotFound(src); - } - Entry* dest_parent = impl->FindParent(dest_parts); - if (dest_parent == nullptr || !dest_parent->is_dir()) { - return PathNotFound(dest); - } - Directory& src_dir = src_parent->as_dir(); - Directory& dest_dir = dest_parent->as_dir(); - DCHECK_GE(src_parts.size(), 1); - DCHECK_GE(dest_parts.size(), 1); - const auto& src_name = src_parts.back(); - const auto& dest_name = dest_parts.back(); - - BinaryOp op{std::move(src_parts), - std::move(dest_parts), - src_dir, - dest_dir, - src_name, - dest_name, - src_dir.Find(src_name), - dest_dir.Find(dest_name)}; - - return op_func(std::move(op)); - } -}; - -} // namespace - -Status MockAzureFileSystem::Move(const std::string& src, const std::string& dest) { - ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); - ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); - - if (src_path == dest_path) { - return Status::OK(); - } - return BinaryOp::Run(impl_.get(), src, dest, [&](const BinaryOp& op) -> Status { - if (op.src_entry == nullptr) { - return PathNotFound(src); - } - if (op.dest_entry != nullptr) { - if (op.dest_entry->is_file() && op.src_entry->is_dir()) { - return Status::IOError("Cannot replace destination '", dest, - "', which is a file, with directory '", src, "'"); - } - if (op.dest_entry->is_dir() && op.src_entry->is_file()) { - return Status::IOError("Cannot replace destination '", dest, - "', which is a directory, with file '", src, "'"); - } - if (op.dest_entry->is_dir() && op.dest_entry->as_dir().entries.size() != 0) { - return Status::IOError("Cannot replace destination '", dest, - "', destination not empty"); - } - } - if (op.src_parts.size() < op.dest_parts.size()) { - // Check if dest is a child of src - auto p = - std::mismatch(op.src_parts.begin(), op.src_parts.end(), op.dest_parts.begin()); - if (p.first == op.src_parts.end()) { - return Status::IOError("Cannot move '", src, "' into child path '", dest, "'"); - } - } - auto path = src_path.path_to_file_parts; - std::unique_ptr new_entry(new Entry(std::move(*op.src_entry))); - new_entry->SetName(op.dest_name); - bool deleted = op.src_dir.DeleteEntry(op.src_name); - DCHECK(deleted); - op.dest_dir.AssignEntry(op.dest_name, std::move(new_entry)); - return Status::OK(); - }); -} - -Status MockAzureFileSystem::CopyFile(const std::string& src, const std::string& dest) { - return BinaryOp::Run(impl_.get(), src, dest, [&](const BinaryOp& op) -> Status { - if (op.src_entry == nullptr) { - return PathNotFound(src); - } - if (!op.src_entry->is_file()) { - return NotAFile(src); - } - if (op.dest_parts.size() == 1) { - return Status::IOError("Cannot copy destination '", dest, - "', which is a container"); - } - if (op.dest_entry != nullptr) { - if (op.dest_entry->is_file() && op.src_entry->is_dir()) { - return Status::IOError("Cannot copy destination '", dest, - "', which is a file, with directory '", src, "'"); - } - if (op.dest_entry->is_dir() && op.src_entry->is_file()) { - return Status::IOError("Cannot copy destination '", dest, - "', which is a directory, with file '", src, "'"); - } - } - if (op.dest_entry != nullptr && op.dest_entry->is_dir()) { - return Status::IOError("Cannot replace destination '", dest, - "', which is a directory"); - } - - // Copy original entry, fix its name - std::unique_ptr new_entry(new Entry(File(op.src_entry->as_file()))); - new_entry->SetName(op.dest_name); - op.dest_dir.AssignEntry(op.dest_name, std::move(new_entry)); - return Status::OK(); - }); -} - -Result> MockAzureFileSystem::OpenInputStream( - const std::string& s) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - if (path.empty()) { - return Status::IOError("Invalid path provided"); - } - if (!(impl_->FileExists(s)).ValueOrDie()) { - return Status::IOError("Invalid path provided"); - } - return impl_->OpenInputReader(s); -} - -Result> MockAzureFileSystem::OpenInputFile( - const std::string& s) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - if (path.empty()) { - return Status::IOError("Invalid path provided"); - } - if (!(impl_->FileExists(s)).ValueOrDie()) { - return Status::IOError("Invalid path provided"); - } - return impl_->OpenInputReader(s); -} - -Result> MockAzureFileSystem::OpenOutputStream( - const std::string& s, const std::shared_ptr& metadata) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - - if (path.empty() || path.path_to_file.empty()) { - return Status::IOError("Invalid path provided"); - } - if (impl_->DirExists(s).ValueOrDie()) { - return Status::IOError("Invalid path provided"); - } - if (path.has_parent()) { - AzurePath parent_path = path.parent(); - if (parent_path.path_to_file.empty()) { - if (!impl_->ContainerExists(parent_path.container).ValueOrDie()) { - return Status::IOError("Cannot write to file '", path.full_path, - "': parent directory does not exist"); - } - } else { - auto exists = impl_->DirExists(parent_path.full_path); - if (!(exists.ValueOrDie())) { - return Status::IOError("Cannot write to file '", path.full_path, - "': parent directory does not exist"); - } - } - } - return impl_->OpenOutputStream(s, /*append=*/false, metadata); -} - -Result> MockAzureFileSystem::OpenAppendStream( - const std::string& s, const std::shared_ptr& metadata) { - ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - - return impl_->OpenOutputStream(s, /*append=*/true, metadata); -} - -bool MockAzureFileSystem::CheckFile(const MockFileInfo& expected) { - auto guard = impl_->lock_guard(); - - return impl_->CheckFile("", impl_->RootDir(), expected); -} - -Status MockAzureFileSystem::CreateFile(const std::string& path, - util::string_view contents, bool recursive) { - RETURN_NOT_OK(ValidatePath(path)); - auto parent = fs::internal::GetAbstractPathParent(path).first; - - if (parent != "") { - RETURN_NOT_OK(CreateDir(parent, recursive)); - } - - ARROW_ASSIGN_OR_RAISE(auto file, OpenOutputStream(path)); - RETURN_NOT_OK(file->Write(contents)); - return file->Close(); -} - -Result> MockAzureFileSystem::Make( - TimePoint current_time, const std::vector& infos) { - auto fs = std::make_shared(current_time); - for (const auto& info : infos) { - switch (info.type()) { - case FileType::Directory: - RETURN_NOT_OK(fs->CreateDir(info.path(), /*recursive*/ true)); - break; - case FileType::File: - RETURN_NOT_OK(fs->CreateFile(info.path(), "", /*recursive*/ true)); - break; - default: - break; - } - } - - return fs; -} -} // namespace internal -} // namespace fs -} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azurefs_mock.h b/cpp/src/arrow/filesystem/azurefs_mock.h deleted file mode 100644 index 5ead83da08192..0000000000000 --- a/cpp/src/arrow/filesystem/azurefs_mock.h +++ /dev/null @@ -1,95 +0,0 @@ -// 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 -#include -#include -#include - -#include "arrow/filesystem/mockfs.h" -#include "arrow/util/string_view.h" -#include "arrow/util/windows_fixup.h" - -namespace arrow { -namespace fs { -namespace internal { - -/// A mock FileSystem implementation that holds its contents in memory. -/// -/// Useful for validating the FileSystem API, writing conformance suite, -/// and bootstrapping FileSystem-based APIs. -class ARROW_EXPORT MockAzureFileSystem : public FileSystem { - public: - explicit MockAzureFileSystem(TimePoint current_time, - const io::IOContext& = io::default_io_context()); - ~MockAzureFileSystem() override; - - std::string type_name() const override { return "mock"; } - - bool Equals(const FileSystem& other) const override; - - // XXX It's not very practical to have to explicitly declare inheritance - // of default overrides. - using FileSystem::GetFileInfo; - Result GetFileInfo(const std::string& path) override; - Result> GetFileInfo(const FileSelector& select) override; - - Status CreateDir(const std::string& path, bool recursive = true) override; - - Status DeleteDir(const std::string& path) override; - Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override; - Status DeleteRootDirContents() override; - - Status DeleteFile(const std::string& path) override; - - Status Move(const std::string& src, const std::string& dest) override; - - Status CopyFile(const std::string& src, const std::string& dest) override; - - Result> OpenInputStream( - const std::string& path) override; - Result> OpenInputFile( - const std::string& path) override; - Result> OpenOutputStream( - const std::string& path, - const std::shared_ptr& metadata = {}) override; - Result> OpenAppendStream( - const std::string& path, - const std::shared_ptr& metadata = {}) override; - - bool CheckFile(const MockFileInfo& expected); - - // Create a File with a content from a string. - Status CreateFile(const std::string& path, util::string_view content, - bool recursive = true); - - // Create a MockAzureFileSystem out of (empty) FileInfo. The content of every - // file is empty and of size 0. All directories will be created recursively. - static Result> Make(TimePoint current_time, - const std::vector& infos); - - class Impl; - - protected: - std::unique_ptr impl_; -}; - -} // namespace internal -} // namespace fs -} // namespace arrow diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 1e683801230ff..38b55cffe9c4b 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. -#include "arrow/filesystem/azurefs_mock.h" +#include "arrow/filesystem/azurefs.h" +#include #include #include #include @@ -38,27 +39,56 @@ namespace internal { class TestAzureFileSystem : public ::testing::Test { public: - TimePoint time_; std::shared_ptr fs_; - - void MakeFileSystem() { fs_ = std::make_shared(time_); } + std::shared_ptr gen2Client_; + AzureOptions options_; + + void MakeFileSystem() { + const std::string& account_name = "devstoreaccount1"; + const std::string& account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="; + options_.account_blob_url = "http://127.0.0.1:10000/devstoreaccount1/"; + options_.account_dfs_url = "http://127.0.0.1:10000/devstoreaccount1/"; + options_.isTestEnabled = true; + options_.storage_credentials_provider = std::make_shared(account_name, account_key); + options_.credentials_kind = AzureCredentialsKind::StorageCredentials; + gen2Client_ = std::make_shared(options_.account_dfs_url, options_.storage_credentials_provider); + auto result = AzureBlobFileSystem::Make(options_); + fs_ = *result; + } void SetUp() override { - time_ = TimePoint(TimePoint::duration(42)); MakeFileSystem(); - ASSERT_OK(fs_->CreateDir("container")); - ASSERT_OK(fs_->CreateDir("empty-container")); - ASSERT_OK(fs_->CreateDir("container2/newdir")); - ASSERT_OK(fs_->CreateDir("container/emptydir")); - ASSERT_OK(fs_->CreateDir("container/somedir")); - ASSERT_OK(fs_->CreateDir("container/somedir/subdir")); - CreateFile(fs_.get(), "container/somedir/subdir/subfile", "sub data"); - CreateFile(fs_.get(), "container/somefile", "some data"); + auto fileSystemClient = gen2Client_->GetFileSystemClient("container"); + fileSystemClient.CreateIfNotExists(); + fileSystemClient = gen2Client_->GetFileSystemClient("empty-container"); + fileSystemClient.CreateIfNotExists(); + auto fileClient = std::make_shared(options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); + std::string s = "some data"; + fileClient->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); } - - bool CheckFile(const MockFileInfo& expected) { - return arrow::internal::checked_pointer_cast(fs_)->CheckFile( - expected); + void TearDown() override { + auto containers = gen2Client_->ListFileSystems(); + for(auto c:containers.FileSystems) { + auto fileSystemClient = gen2Client_->GetFileSystemClient(c.Name); + fileSystemClient.DeleteIfExists(); + } + } + void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* client, const std::string& container, + const std::string& path_to_file, const std::string& expected) { + auto pathClient_ = std::make_shared(client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); + auto size = pathClient_->GetProperties().Value.FileSize; + if (size == 0) { + return; + } + auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); + Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; + Azure::Core::Http::HttpRange range; + range.Offset = 0; + range.Length = size; + downloadOptions.Range = Azure::Nullable(range); + auto fileClient_ = std::make_shared(client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); + auto result = fileClient_->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), size, downloadOptions).Value; + buf->get()->Equals(Buffer(const_cast(reinterpret_cast(&expected[0])), expected.size())); } }; @@ -74,174 +104,94 @@ TEST_F(TestAzureFileSystem, CreateDir) { ASSERT_RAISES(IOError, fs_->CreateDir("")); - // New "directory", true - AssertFileInfo(fs_.get(), "container/newdir", FileType::NotFound); - ASSERT_OK(fs_->CreateDir("container/newdir", true)); - AssertFileInfo(fs_.get(), "container/newdir", FileType::Directory); - - // New "directory", false - AssertFileInfo(fs_.get(), "container/newdir1", FileType::NotFound); - ASSERT_OK(fs_->CreateDir("container/newdir1", false)); - AssertFileInfo(fs_.get(), "container/newdir1", FileType::Directory); - - // Existing "directory", true - AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); - ASSERT_OK(fs_->CreateDir("container/somedir", true)); - AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); - - // Existing "directory", false - AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); - ASSERT_OK(fs_->CreateDir("container/somedir", false)); - AssertFileInfo(fs_.get(), "container/somedir", FileType::Directory); - // Existing "file", should fail ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); - // C/D/D - AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); - ASSERT_OK(fs_->CreateDir("container/somedir/subdir")); - AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); - - auto res = fs_->OpenOutputStream("container/somedir/base.txt"); - ASSERT_OK(res->get()->Write("Changed the data")); - - // C/D/F - AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); - ASSERT_RAISES(IOError, fs_->CreateDir("container/somedir/base.txt")); - AssertFileInfo(fs_.get(), "container/somedir/base.txt", FileType::File); - - // New "directory",Parent dir not exists, false + // directory, false ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", false)); - // New "directory",Parent dir not exists, true - ASSERT_OK(fs_->CreateDir("container/newdir/newsub/newsubsub", true)); - AssertFileInfo(fs_.get(), "container/newdir/newsub", FileType::Directory); - AssertFileInfo(fs_.get(), "container/newdir/newsub/newsubsub", FileType::Directory); - - // New "directory",Container not exists, false - ASSERT_RAISES(IOError, fs_->CreateDir("container4/newdir", false)); - - // New "directory",Container not exists, true - ASSERT_OK(fs_->CreateDir("container4/newdir", true)); - AssertFileInfo(fs_.get(), "container4", FileType::Directory); - AssertFileInfo(fs_.get(), "container4/newdir", FileType::Directory); + // directory, true + ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", true)); } TEST_F(TestAzureFileSystem, DeleteDir) { - FileSelector select; - select.base_dir = "container2"; - std::vector infos; - // Container - ASSERT_OK(fs_->DeleteDir("empty-container")); - AssertFileInfo(fs_.get(), "empty-container", FileType::NotFound); + ASSERT_OK(fs_->DeleteDir("container")); + AssertFileInfo(fs_.get(), "container", FileType::NotFound); - // Nonexistent Container + // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); - // C/D - ASSERT_OK(fs_->DeleteDir("container2/newdir")); - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - - // C/ND - AssertFileInfo(fs_.get(), "container2/newdir1", FileType::NotFound); - ASSERT_RAISES(IOError, fs_->DeleteDir("container2/newdir1")); - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - // C/F ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); - // C/D/D - ASSERT_OK(fs_->DeleteDir("container/somedir/subdir")); - - // C/D/F - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir/subdir/subfile")); - + // C/NF + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile19")); + // C/ND/D ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container6/somedir")); + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); } TEST_F(TestAzureFileSystem, DeleteFile) { - FileSelector select; - select.base_dir = "container2"; - std::vector infos; - // Container - ASSERT_RAISES(IOError, fs_->DeleteFile("container2")); + ASSERT_RAISES(IOError, fs_->DeleteFile("container")); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteFile("")); - - // C/D - ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptydir")); - - // C/ND - ASSERT_RAISES(IOError, fs_->DeleteFile("container/emptydir1")); - + // C/F ASSERT_OK(fs_->DeleteFile("container/somefile")); - + // C/NF ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); // C/D/D ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); - // C/D/F - AssertFileInfo(fs_.get(), "container/somedir/subdir/subfile", FileType::File); - ASSERT_OK(fs_->DeleteFile("container/somedir/subdir/subfile")); - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir3/base")); + ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); // NC/D - ASSERT_RAISES(IOError, fs_->DeleteFile("container7/somedir")); + ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); } TEST_F(TestAzureFileSystem, GetFileInfo) { - // Containers + //Containers AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); - AssertFileInfo(fs_.get(), "", FileType::Directory); + AssertFileInfo(fs_.get(), "", FileType::Directory); - // "Directories" - AssertFileInfo(fs_.get(), "container/emptydir", FileType::Directory); - AssertFileInfo(fs_.get(), "container/emptydir1", FileType::NotFound); + auto res = fs_->OpenOutputStream("container/base.txt"); + ASSERT_OK(res->get()->Write("Base data")); // "Files" - AssertFileInfo(fs_.get(), "container/somefile", FileType::File); - AssertFileInfo(fs_.get(), "container/somefile1", FileType::NotFound); - - // "Directories" - AssertFileInfo(fs_.get(), "container/somedir/subdir", FileType::Directory); - AssertFileInfo(fs_.get(), "container/somedir/subdir/subfile", FileType::File); + AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); + AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); // "Directories" - AssertFileInfo(fs_.get(), "container/somedir45/subdir", FileType::NotFound); - AssertFileInfo(fs_.get(), "containe23r/somedir/subdir/subfile", FileType::NotFound); + AssertFileInfo(fs_.get(), "container/non-existentdir/subdir", FileType::NotFound); + AssertFileInfo(fs_.get(), "nonexistent-container/somedir/subdir/subfile", FileType::NotFound); } -TEST_F(TestAzureFileSystem, GetFileInfoSelector) { +TEST_F(TestAzureFileSystem, GetFileInfoSelector) { FileSelector select; std::vector infos; // Non-empty container select.base_dir = "container"; ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 3); + ASSERT_EQ(infos.size(), 1); // Nonexistent container select.base_dir = "nonexistent-container"; @@ -254,36 +204,14 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { // Root dir select.base_dir = ""; ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 3); - - // C/D - select.base_dir = "container/somedir"; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 1); - - // C/ND - select.base_dir = "container/sgsgs"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; + ASSERT_EQ(infos.size(), 2); // C/F select.base_dir = "container/somefile"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - // C/D/D - select.base_dir = "container/somedir/subdir"; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 1); - - // C/F - select.base_dir = "container/somedir/subdir/subfile"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - // C/ND/D - select.base_dir = "container/ahsh/agsg"; + select.base_dir = "container/non-existentdir/non-existentsubdir"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); select.allow_not_found = true; ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); @@ -291,7 +219,7 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { select.allow_not_found = false; // NC/D - select.base_dir = "nonexistent-container/agshhs"; + select.base_dir = "nonexistent-container/non-existentdir"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); select.allow_not_found = true; ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); @@ -299,61 +227,46 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { select.allow_not_found = false; } -TEST_F(TestAzureFileSystem, Move) { - ASSERT_RAISES(IOError, fs_->Move("container", "container/nshhd")); - ASSERT_OK(fs_->CreateDir("container/newdir/newsub/newsubsub", true)); - ASSERT_RAISES(IOError, - fs_->Move("container/somedir/subdir", "container/newdir/newsub")); - ASSERT_OK(fs_->Move("container/newdir/newsub", "container/emptydir")); - ASSERT_OK(fs_->Move("container/emptydir", "container/emptydir1")); - ASSERT_OK(fs_->Move("container/emptydir1", "container/emptydir")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/somefile")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/ahsh/gssjd")); +TEST_F(TestAzureFileSystem, Move) { + ASSERT_RAISES(IOError, fs_->Move("container", "container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "nonexistent-container/non-existentdir")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); - ASSERT_OK(fs_->Move("container/somedir/subdir/subfile", "container/somefile")); - ASSERT_OK(fs_->Move("container/somefile", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/ahsh/gssjd")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "containerqw/ghdj")); - ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/gshh")); + auto res = fs_->OpenOutputStream("container/somefile"); + ASSERT_OK(res->get()->Write("Changed the data")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); + ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/non-existentdir")); } TEST_F(TestAzureFileSystem, CopyFile) { - // "File" ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container23/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("nonexistent-container/somedir/subdir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container27/hshj")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "container27/hshj")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "nonexistent-container/non-existentdir")); - ASSERT_OK(fs_->CopyFile("container/somefile", "container/somedir/subdir/subfile")); - ASSERT_OK(fs_->CopyFile("container/somefile", "container/somefile3")); + ASSERT_OK(fs_->CopyFile("container/somefile", "container/base.txt")); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "nonexistent-container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/somedir/subdir")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container/ahsj/ggws")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir/subdir/subfile", "container27/hshj")); - ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile")); - ASSERT_OK(fs_->DeleteFile("container/somefile3")); - ASSERT_OK(fs_->CopyFile("container/somedir/subdir/subfile", "container/somefile3")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "nonexistent-container/non-existentdir")); } TEST_F(TestAzureFileSystem, OpenInputStream) { @@ -361,13 +274,13 @@ TEST_F(TestAzureFileSystem, OpenInputStream) { std::shared_ptr buf; ASSERT_RAISES(IOError, fs_->OpenInputStream("container")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenInputStream("")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/non-existentdir")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container526/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("container/non-existentdir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputStream("nonexistent-container/somedir")); // "Files" ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); @@ -379,15 +292,6 @@ TEST_F(TestAzureFileSystem, OpenInputStream) { AssertBufferEqual(*buf, "ta"); ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); AssertBufferEqual(*buf, ""); - - CreateFile(fs_.get(), "container/subfile", "sub data"); - - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/subfile")); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); - AssertBufferEqual(*buf, "sub data"); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(100)); - AssertBufferEqual(*buf, ""); - ASSERT_OK(stream->Close()); } TEST_F(TestAzureFileSystem, OpenInputFile) { @@ -395,13 +299,13 @@ TEST_F(TestAzureFileSystem, OpenInputFile) { std::shared_ptr buf; ASSERT_RAISES(IOError, fs_->OpenInputFile("container")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container263")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenInputFile("")); ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/sjdjd")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/non-existentdir")); ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container526/somedir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("container/non-existentdir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenInputFile("nonexistent-container/somedir")); // "Files" ASSERT_OK_AND_ASSIGN(file, fs_->OpenInputFile("container/somefile")); @@ -441,23 +345,22 @@ TEST_F(TestAzureFileSystem, OpenOutputStream) { std::shared_ptr stream; ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container263")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenOutputStream("")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir")); ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/shjdj/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container526/somedir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/non-existentdir/subdir")); + ASSERT_RAISES(IOError, fs_->OpenOutputStream("nonexistent-container/somedir")); // Create new empty file ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile1", time_, ""})); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); // Create new file with 1 small write ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); ASSERT_OK(stream->Write("some data")); ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile2", time_, "some data"})); + AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); // Create new file with 3 small writes ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); @@ -465,7 +368,7 @@ TEST_F(TestAzureFileSystem, OpenOutputStream) { ASSERT_OK(stream->Write("")); ASSERT_OK(stream->Write("new data")); ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile3", time_, "some new data"})); + AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); // Create new file with some large writes std::string s1, s2, s3, s4, s5, expected; @@ -483,57 +386,38 @@ TEST_F(TestAzureFileSystem, OpenOutputStream) { input.back() = 'x'; } ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile4", time_, expected})); + AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); // Overwrite ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Write("overwritten data")); ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile1", time_, "overwritten data"})); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); // Overwrite and make empty ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - ASSERT_TRUE(CheckFile({"container/newfile1", time_, ""})); + AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); } TEST_F(TestAzureFileSystem, DeleteDirContents) { - FileSelector select; - select.base_dir = "container2/newdir"; - std::vector infos; - // Container - ASSERT_OK(fs_->DeleteDirContents("container2")); - AssertFileInfo(fs_.get(), "container2", FileType::Directory); + ASSERT_OK(fs_->DeleteDirContents("container")); + AssertFileInfo(fs_.get(), "container", FileType::Directory); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - - ASSERT_OK(fs_->CreateDir("container2/newdir/subdir", true)); - - // C/D - ASSERT_OK(fs_->DeleteDirContents("container2/newdir")); - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - - // C/ND - AssertFileInfo(fs_.get(), "container2/newdir1", FileType::NotFound); - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container2/newdir1")); - + // C/F + auto res = fs_->OpenOutputStream("container/somefile"); + ASSERT_OK(res->get()->Write("some data")); ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); AssertFileInfo(fs_.get(), "container/somefile", FileType::File); - // C/D/D - - ASSERT_OK(fs_->DeleteDirContents("container/somedir/subdir")); - select.base_dir = "container/somedir/subdir"; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - // C/ND/D ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somedir3/base")); @@ -543,4 +427,4 @@ TEST_F(TestAzureFileSystem, DeleteDirContents) { } // namespace internal } // namespace fs -} // namespace arrow +} // namespace arrow \ No newline at end of file From ca9a6fc53bf55f7bf2771f3715910db565eadc93 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Sat, 25 Jun 2022 22:19:51 +0530 Subject: [PATCH 17/34] Added azurefs_objlib --- cpp/src/arrow/CMakeLists.txt | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index 4041a399cb193..b46ef27bc9bf9 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -471,9 +471,14 @@ if(ARROW_FILESYSTEM) if(ARROW_AZURE) set(AZURE_SRCS filesystem/azurefs.cc) - set_source_files_properties(filesystem/azurefs.cc filesystem/azurefs_mock.cc + set_source_files_properties(filesystem/azurefs.cc PROPERTIES SKIP_PRECOMPILE_HEADERS ON SKIP_UNITY_BUILD_INCLUSION ON) + add_library(azurefs_objlib OBJECT ${AZURE_SRCS}) + target_link_libraries(azurefs_objlib PUBLIC ${ARROW_AZURE_LINK_LIBS}) + + set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) + list(APPEND ARROW_LINK_LIBS ${ARROW_AZURE_LINK_LIBS} azurefs_objlib) endif() if(ARROW_GCS) list(APPEND ARROW_SRCS filesystem/gcsfs.cc filesystem/gcsfs_internal.cc) @@ -554,12 +559,6 @@ if(${CMAKE_SYSTEM_NAME} STREQUAL "Linux" AND ${CMAKE_SYSTEM_PROCESSOR} MATCHES " string(APPEND ARROW_PC_LIBS_PRIVATE " -latomic") endif() -add_library(azurefs_objlib OBJECT ${AZURE_SRCS}) -target_link_libraries(azurefs_objlib PUBLIC ${ARROW_AZURE_LINK_LIBS}) - -set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) -list(APPEND ARROW_LINK_LIBS ${ARROW_AZURE_LINK_LIBS} azurefs_objlib) - add_arrow_lib(arrow CMAKE_PACKAGE_NAME Arrow From f067ba94403f8af883a8a55893a72b6dea93437e Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 27 Jun 2022 02:59:02 +0530 Subject: [PATCH 18/34] Reverting azure object library changes --- .github/workflows/cpp.yml | 6 ++++-- ci/scripts/install_azurite.sh | 21 +++++++++++++++------ cpp/CMakeLists.txt | 6 +++--- cpp/cmake_modules/SetupCxxFlags.cmake | 10 +++++++--- cpp/src/arrow/CMakeLists.txt | 7 +------ cpp/src/arrow/filesystem/CMakeLists.txt | 2 -- 6 files changed, 30 insertions(+), 22 deletions(-) diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index 2d6f720a3bb2b..49574e6fa3658 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -163,7 +163,9 @@ jobs: run: ci/scripts/install_gcs_testbench.sh default - name: Install Azurite Storage Emulator shell: bash - run: ci/scripts/install_azurite.sh $(pwd) + run: | + chmod +x ci/scripts/install_azurite.sh + ci/scripts/install_azurite.sh $(pwd) - name: Setup ccache run: | ci/scripts/ccache_setup.sh @@ -272,7 +274,7 @@ jobs: - 32 - 64 env: - ARROW_AZURE: ON + ARROW_AZURE: OFF ARROW_BUILD_SHARED: ON ARROW_BUILD_STATIC: OFF ARROW_BUILD_TESTS: ON diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh index 9af468603bc0e..5baf40d6bf334 100644 --- a/ci/scripts/install_azurite.sh +++ b/ci/scripts/install_azurite.sh @@ -19,13 +19,22 @@ set -e -sudo apt-get -y install nodejs +if [[ "$OSTYPE" == "darwin20" ]]; then + brew install node + npm install -g azurite + which azurite +elif [[ "$OSTYPE" == "msys" ]]; then + choco install nodejs.install + npm install -g azurite +else + apt-get -y install nodejs + npm install -g azurite + which azurite +fi echo "node version = `node --version`" -sudo npm install -g azurite -AZURITE_DIR=${0}/azurite -mkdir $AZURITE_DIR -which azurite echo "azurite version = `azurite --version`" +AZURITE_DIR=${1}/azurite +mkdir $AZURITE_DIR # Start azurite -azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log +azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log & diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 969355b89695b..89159ba00b65d 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -cmake_minimum_required(VERSION 3.12) +cmake_minimum_required(VERSION 3.5) message(STATUS "Building using CMake version: ${CMAKE_VERSION}") # Compiler id for Apple Clang is now AppleClang. @@ -714,7 +714,6 @@ endif() set(ARROW_LINK_LIBS arrow::flatbuffers arrow::hadoop) set(ARROW_STATIC_LINK_LIBS arrow::flatbuffers arrow::hadoop) set(ARROW_STATIC_INSTALL_INTERFACE_LIBS) -set(ARROW_AZURE_LINK_LIBS) if(ARROW_USE_BOOST) list(APPEND ARROW_LINK_LIBS Boost::headers) @@ -818,7 +817,8 @@ if(ARROW_WITH_OPENTELEMETRY) endif() if(ARROW_AZURE) - list(APPEND ARROW_AZURE_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) endif() if(ARROW_WITH_UTF8PROC) diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index d3a2a1a2d256a..cdaafe379b221 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -118,12 +118,16 @@ if(NOT DEFINED CMAKE_C_STANDARD) set(CMAKE_C_STANDARD 11) endif() -# This ensures that things like c++11 get passed correctly +# This ensures that things like c++11/c++14 get passed correctly if(NOT DEFINED CMAKE_CXX_STANDARD) - set(CMAKE_CXX_STANDARD 11) + if(ARROW_AZURE) + set(CMAKE_CXX_STANDARD 14) + else() + set(CMAKE_CXX_STANDARD 11) + endif() endif() -# We require a C++11 compliant compiler +# We require a C++11/14 compliant compiler set(CMAKE_CXX_STANDARD_REQUIRED ON) # ARROW-6848: Do not use GNU (or other CXX) extensions diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt index b46ef27bc9bf9..cd74cb15a4b74 100644 --- a/cpp/src/arrow/CMakeLists.txt +++ b/cpp/src/arrow/CMakeLists.txt @@ -470,15 +470,10 @@ if(ARROW_FILESYSTEM) filesystem/util_internal.cc) if(ARROW_AZURE) - set(AZURE_SRCS filesystem/azurefs.cc) + list(APPEND ARROW_SRCS filesystem/azurefs.cc) set_source_files_properties(filesystem/azurefs.cc PROPERTIES SKIP_PRECOMPILE_HEADERS ON SKIP_UNITY_BUILD_INCLUSION ON) - add_library(azurefs_objlib OBJECT ${AZURE_SRCS}) - target_link_libraries(azurefs_objlib PUBLIC ${ARROW_AZURE_LINK_LIBS}) - - set_target_properties(azurefs_objlib PROPERTIES CXX_STANDARD 14 CXX_STANDARD_REQUIRED ON) - list(APPEND ARROW_LINK_LIBS ${ARROW_AZURE_LINK_LIBS} azurefs_objlib) endif() if(ARROW_GCS) list(APPEND ARROW_SRCS filesystem/gcsfs.cc filesystem/gcsfs_internal.cc) diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index 9a42ecc89f327..6d0f8dc3bb53b 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -39,8 +39,6 @@ endif() if(ARROW_AZURE) add_arrow_test(azurefs_test EXTRA_LABELS filesystem) - set_target_properties(arrow-azurefs-test PROPERTIES CXX_STANDARD 14 - CXX_STANDARD_REQUIRED ON) endif() if(ARROW_S3) From 1f2672547ad8e965b629756708cdb2e25441d319 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 27 Jun 2022 13:39:30 +0530 Subject: [PATCH 19/34] Added permissions to install_azurite.sh --- ci/docker/ubuntu-20.04-cpp.dockerfile | 4 +++- ci/docker/ubuntu-22.04-cpp.dockerfile | 4 +++- ci/scripts/install_azurite.sh | 3 ++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index 2daa2c3222a2d..376d2447cdc68 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -113,7 +113,9 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN /arrow/ci/scripts/install_azurite.sh /usr/local +RUN \ + chmod +x /arrow/ci/scripts/install_azurite.sh && \ + /arrow/ci/scripts/install_azurite.sh /usr/local COPY ci/scripts/install_ceph.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_ceph.sh diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index e105a8f47006f..db291f85347e0 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -144,7 +144,9 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN /arrow/ci/scripts/install_azurite.sh /usr/local +RUN \ + chmod +x /arrow/ci/scripts/install_azurite.sh && \ + /arrow/ci/scripts/install_azurite.sh /usr/local # Prioritize system packages and local installation # The following dependencies will be downloaded due to missing/invalid packages diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh index 5baf40d6bf334..900ebf5ece97f 100644 --- a/ci/scripts/install_azurite.sh +++ b/ci/scripts/install_azurite.sh @@ -27,7 +27,8 @@ elif [[ "$OSTYPE" == "msys" ]]; then choco install nodejs.install npm install -g azurite else - apt-get -y install nodejs + apt-get update + apt-get -y install npm npm install -g azurite which azurite fi From c16f853659893ef3feae8a20fb925157b3ca01ce Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 10:46:29 +0900 Subject: [PATCH 20/34] chmod +x ci/scripts/install_azurite.sh --- .github/workflows/cpp.yml | 4 +--- ci/docker/ubuntu-20.04-cpp.dockerfile | 4 +--- ci/docker/ubuntu-22.04-cpp.dockerfile | 4 +--- ci/scripts/install_azurite.sh | 0 4 files changed, 3 insertions(+), 9 deletions(-) mode change 100644 => 100755 ci/scripts/install_azurite.sh diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index 49574e6fa3658..f2b2da6f158ab 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -163,9 +163,7 @@ jobs: run: ci/scripts/install_gcs_testbench.sh default - name: Install Azurite Storage Emulator shell: bash - run: | - chmod +x ci/scripts/install_azurite.sh - ci/scripts/install_azurite.sh $(pwd) + run: ci/scripts/install_azurite.sh $(pwd) - name: Setup ccache run: | ci/scripts/ccache_setup.sh diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index 376d2447cdc68..2daa2c3222a2d 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -113,9 +113,7 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN \ - chmod +x /arrow/ci/scripts/install_azurite.sh && \ - /arrow/ci/scripts/install_azurite.sh /usr/local +RUN /arrow/ci/scripts/install_azurite.sh /usr/local COPY ci/scripts/install_ceph.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_ceph.sh diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index db291f85347e0..e105a8f47006f 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -144,9 +144,7 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN \ - chmod +x /arrow/ci/scripts/install_azurite.sh && \ - /arrow/ci/scripts/install_azurite.sh /usr/local +RUN /arrow/ci/scripts/install_azurite.sh /usr/local # Prioritize system packages and local installation # The following dependencies will be downloaded due to missing/invalid packages diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh old mode 100644 new mode 100755 From 14267c2d1d3f82523f1857f36539231b01dd55e7 Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 14:04:12 +0900 Subject: [PATCH 21/34] Don't specify CMAKE_CXX_STANDARD by default --- ci/scripts/cpp_build.sh | 2 +- cpp/cmake_modules/SetupCxxFlags.cmake | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ci/scripts/cpp_build.sh b/ci/scripts/cpp_build.sh index c3a8d8887078c..31f8cebb931ff 100755 --- a/ci/scripts/cpp_build.sh +++ b/ci/scripts/cpp_build.sh @@ -142,7 +142,7 @@ cmake \ -DCMAKE_VERBOSE_MAKEFILE=${CMAKE_VERBOSE_MAKEFILE:-OFF} \ -DCMAKE_C_FLAGS="${CFLAGS:-}" \ -DCMAKE_CXX_FLAGS="${CXXFLAGS:-}" \ - -DCMAKE_CXX_STANDARD="${CMAKE_CXX_STANDARD:-11}" \ + -DCMAKE_CXX_STANDARD="${CMAKE_CXX_STANDARD:-}" \ -DCMAKE_INSTALL_LIBDIR=${CMAKE_INSTALL_LIBDIR:-lib} \ -DCMAKE_INSTALL_PREFIX=${CMAKE_INSTALL_PREFIX:-${ARROW_HOME}} \ -DCMAKE_UNITY_BUILD=${CMAKE_UNITY_BUILD:-OFF} \ diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index cdaafe379b221..29b7325e1f52d 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -114,12 +114,12 @@ elseif(ARROW_CPU_FLAG STREQUAL "armv8") endif() # Support C11 -if(NOT DEFINED CMAKE_C_STANDARD) +if(CMAKE_C_STANDARD STREQUAL "") set(CMAKE_C_STANDARD 11) endif() # This ensures that things like c++11/c++14 get passed correctly -if(NOT DEFINED CMAKE_CXX_STANDARD) +if(CMAKE_CXX_STANDARD STREQUAL "") if(ARROW_AZURE) set(CMAKE_CXX_STANDARD 14) else() From 11ce11fb382788deccc876365a9df0eefdce3774 Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 14:07:04 +0900 Subject: [PATCH 22/34] Fix system detection --- ci/scripts/install_azurite.sh | 36 +++++++++++++++++++---------------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh index 900ebf5ece97f..0a396f5e0b4c6 100755 --- a/ci/scripts/install_azurite.sh +++ b/ci/scripts/install_azurite.sh @@ -19,23 +19,27 @@ set -e -if [[ "$OSTYPE" == "darwin20" ]]; then - brew install node - npm install -g azurite - which azurite -elif [[ "$OSTYPE" == "msys" ]]; then - choco install nodejs.install - npm install -g azurite -else - apt-get update - apt-get -y install npm - npm install -g azurite - which azurite -fi -echo "node version = `node --version`" -echo "azurite version = `azurite --version`" +case "$(uname)" in + Darwin) + brew install node + npm install -g azurite + which azurite + ;; + MINGW*) + choco install nodejs.install + npm install -g azurite + ;; + Linux) + apt-get update + apt-get -y install npm + npm install -g azurite + which azurite + ;; +esac +echo "node version = $(node --version)" +echo "azurite version = $(azurite --version)" AZURITE_DIR=${1}/azurite mkdir $AZURITE_DIR # Start azurite -azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log & +azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log & From a428a2b8075d7da5299ab804d824bcfc1fe1552c Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 14:12:22 +0900 Subject: [PATCH 23/34] Fix syntax --- cpp/cmake_modules/SetupCxxFlags.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/cmake_modules/SetupCxxFlags.cmake b/cpp/cmake_modules/SetupCxxFlags.cmake index 29b7325e1f52d..9fdd197429c71 100644 --- a/cpp/cmake_modules/SetupCxxFlags.cmake +++ b/cpp/cmake_modules/SetupCxxFlags.cmake @@ -114,12 +114,12 @@ elseif(ARROW_CPU_FLAG STREQUAL "armv8") endif() # Support C11 -if(CMAKE_C_STANDARD STREQUAL "") +if("${CMAKE_C_STANDARD}" STREQUAL "") set(CMAKE_C_STANDARD 11) endif() # This ensures that things like c++11/c++14 get passed correctly -if(CMAKE_CXX_STANDARD STREQUAL "") +if("${CMAKE_CXX_STANDARD}" STREQUAL "") if(ARROW_AZURE) set(CMAKE_CXX_STANDARD 14) else() From a62d10404f844d5541cc397baee3e351d90c30f6 Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 14:13:22 +0900 Subject: [PATCH 24/34] Fix style --- cpp/src/arrow/filesystem/azurefs_test.cc | 138 +++++++++++++++-------- 1 file changed, 89 insertions(+), 49 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 38b55cffe9c4b..e50228f95fe3d 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -17,8 +17,8 @@ #include "arrow/filesystem/azurefs.h" -#include #include +#include #include #include @@ -43,15 +43,21 @@ class TestAzureFileSystem : public ::testing::Test { std::shared_ptr gen2Client_; AzureOptions options_; - void MakeFileSystem() { + void MakeFileSystem() { const std::string& account_name = "devstoreaccount1"; - const std::string& account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="; + const std::string& account_key = + "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/" + "KBHBeksoGMGw=="; options_.account_blob_url = "http://127.0.0.1:10000/devstoreaccount1/"; options_.account_dfs_url = "http://127.0.0.1:10000/devstoreaccount1/"; options_.isTestEnabled = true; - options_.storage_credentials_provider = std::make_shared(account_name, account_key); + options_.storage_credentials_provider = + std::make_shared(account_name, + account_key); options_.credentials_kind = AzureCredentialsKind::StorageCredentials; - gen2Client_ = std::make_shared(options_.account_dfs_url, options_.storage_credentials_provider); + gen2Client_ = + std::make_shared( + options_.account_dfs_url, options_.storage_credentials_provider); auto result = AzureBlobFileSystem::Make(options_); fs_ = *result; } @@ -62,20 +68,29 @@ class TestAzureFileSystem : public ::testing::Test { fileSystemClient.CreateIfNotExists(); fileSystemClient = gen2Client_->GetFileSystemClient("empty-container"); fileSystemClient.CreateIfNotExists(); - auto fileClient = std::make_shared(options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); + auto fileClient = + std::make_shared( + options_.account_blob_url + "container/somefile", + options_.storage_credentials_provider); std::string s = "some data"; - fileClient->UploadFrom(const_cast(reinterpret_cast(&s[0])), s.size()); + fileClient->UploadFrom(const_cast(reinterpret_cast(&s[0])), + s.size()); } void TearDown() override { auto containers = gen2Client_->ListFileSystems(); - for(auto c:containers.FileSystems) { + for (auto c : containers.FileSystems) { auto fileSystemClient = gen2Client_->GetFileSystemClient(c.Name); fileSystemClient.DeleteIfExists(); } } - void AssertObjectContents(Azure::Storage::Files::DataLake::DataLakeServiceClient* client, const std::string& container, - const std::string& path_to_file, const std::string& expected) { - auto pathClient_ = std::make_shared(client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); + void AssertObjectContents( + Azure::Storage::Files::DataLake::DataLakeServiceClient* client, + const std::string& container, const std::string& path_to_file, + const std::string& expected) { + auto pathClient_ = + std::make_shared( + client->GetUrl() + container + "/" + path_to_file, + options_.storage_credentials_provider); auto size = pathClient_->GetProperties().Value.FileSize; if (size == 0) { return; @@ -86,9 +101,17 @@ class TestAzureFileSystem : public ::testing::Test { range.Offset = 0; range.Length = size; downloadOptions.Range = Azure::Nullable(range); - auto fileClient_ = std::make_shared(client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); - auto result = fileClient_->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), size, downloadOptions).Value; - buf->get()->Equals(Buffer(const_cast(reinterpret_cast(&expected[0])), expected.size())); + auto fileClient_ = + std::make_shared( + client->GetUrl() + container + "/" + path_to_file, + options_.storage_credentials_provider); + auto result = fileClient_ + ->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), + size, downloadOptions) + .Value; + buf->get()->Equals( + Buffer(const_cast(reinterpret_cast(&expected[0])), + expected.size())); } }; @@ -117,13 +140,13 @@ TEST_F(TestAzureFileSystem, CreateDir) { TEST_F(TestAzureFileSystem, DeleteDir) { // Container ASSERT_OK(fs_->DeleteDir("container")); - AssertFileInfo(fs_.get(), "container", FileType::NotFound); + AssertFileInfo(fs_.get(), "container", FileType::NotFound); - // Nonexistent Container + // Nonexistent Container ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); // C/F @@ -131,7 +154,7 @@ TEST_F(TestAzureFileSystem, DeleteDir) { // C/NF ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile19")); - + // C/ND/D ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); @@ -143,15 +166,15 @@ TEST_F(TestAzureFileSystem, DeleteFile) { // Container ASSERT_RAISES(IOError, fs_->DeleteFile("container")); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteFile("")); - + // C/F ASSERT_OK(fs_->DeleteFile("container/somefile")); - + // C/NF ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); @@ -166,11 +189,11 @@ TEST_F(TestAzureFileSystem, DeleteFile) { } TEST_F(TestAzureFileSystem, GetFileInfo) { - //Containers + // Containers AssertFileInfo(fs_.get(), "container", FileType::Directory); AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); - AssertFileInfo(fs_.get(), "", FileType::Directory); + AssertFileInfo(fs_.get(), "", FileType::Directory); auto res = fs_->OpenOutputStream("container/base.txt"); ASSERT_OK(res->get()->Write("Base data")); @@ -181,10 +204,11 @@ TEST_F(TestAzureFileSystem, GetFileInfo) { // "Directories" AssertFileInfo(fs_.get(), "container/non-existentdir/subdir", FileType::NotFound); - AssertFileInfo(fs_.get(), "nonexistent-container/somedir/subdir/subfile", FileType::NotFound); + AssertFileInfo(fs_.get(), "nonexistent-container/somedir/subdir/subfile", + FileType::NotFound); } -TEST_F(TestAzureFileSystem, GetFileInfoSelector) { +TEST_F(TestAzureFileSystem, GetFileInfoSelector) { FileSelector select; std::vector infos; @@ -227,19 +251,24 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { select.allow_not_found = false; } -TEST_F(TestAzureFileSystem, Move) { +TEST_F(TestAzureFileSystem, Move) { ASSERT_RAISES(IOError, fs_->Move("container", "container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); + ASSERT_RAISES(IOError, + fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->Move("container/emptydir", + "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, + fs_->Move("container/emptydir", "nonexistent-container/non-existentdir")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); auto res = fs_->OpenOutputStream("container/somefile"); ASSERT_OK(res->get()->Write("Changed the data")); ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->Move("container/base.txt", + "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, + fs_->Move("container/base.txt", "nonexistent-container/non-existentdir")); ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/non-existentdir")); } @@ -247,26 +276,37 @@ TEST_F(TestAzureFileSystem, CopyFile) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("nonexistent-container/somedir/subdir", "container/newfile")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir22/subdir", "container/newfile")); + ASSERT_RAISES(IOError, fs_->CopyFile("nonexistent-container/somedir/subdir", + "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "nonexistent-container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", + "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", + "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", + "nonexistent-container/non-existentdir")); ASSERT_OK(fs_->CopyFile("container/somefile", "container/base.txt")); ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "nonexistent-container")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "nonexistent-container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "nonexistent-container/non-existentdir")); + ASSERT_RAISES(IOError, + fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/somedir/subdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "container/non-existentdir/non-existentsubdir")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", + "nonexistent-container/non-existentdir")); } TEST_F(TestAzureFileSystem, OpenInputStream) { @@ -403,15 +443,15 @@ TEST_F(TestAzureFileSystem, OpenOutputStream) { TEST_F(TestAzureFileSystem, DeleteDirContents) { // Container ASSERT_OK(fs_->DeleteDirContents("container")); - AssertFileInfo(fs_.get(), "container", FileType::Directory); + AssertFileInfo(fs_.get(), "container", FileType::Directory); - // Nonexistent Container + // Nonexistent Container ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - // root + // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - + // C/F auto res = fs_->OpenOutputStream("container/somefile"); ASSERT_OK(res->get()->Write("some data")); From 488e2236c6412f977bc5df7e34a0145b437ed6e4 Mon Sep 17 00:00:00 2001 From: Sutou Kouhei Date: Tue, 28 Jun 2022 16:38:25 +0900 Subject: [PATCH 25/34] Fix style --- cpp/src/arrow/filesystem/azurefs_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index e50228f95fe3d..751192ffa9276 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -467,4 +467,4 @@ TEST_F(TestAzureFileSystem, DeleteDirContents) { } // namespace internal } // namespace fs -} // namespace arrow \ No newline at end of file +} // namespace arrow From 3831a8838530a7484adb74c15dfd6e14ddebde6a Mon Sep 17 00:00:00 2001 From: shefali singh Date: Tue, 28 Jun 2022 16:27:29 +0530 Subject: [PATCH 26/34] Running azurite through boost::process --- ci/scripts/install_azurite.sh | 7 +-- cpp/src/arrow/filesystem/CMakeLists.txt | 7 ++- cpp/src/arrow/filesystem/azurefs_test.cc | 74 ++++++++++++++++++++++-- 3 files changed, 75 insertions(+), 13 deletions(-) diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh index 0a396f5e0b4c6..a76cd8b01765d 100755 --- a/ci/scripts/install_azurite.sh +++ b/ci/scripts/install_azurite.sh @@ -37,9 +37,4 @@ case "$(uname)" in ;; esac echo "node version = $(node --version)" -echo "azurite version = $(azurite --version)" -AZURITE_DIR=${1}/azurite -mkdir $AZURITE_DIR - -# Start azurite -azurite --silent --location $AZURITE_DIR --debug $AZURITE_DIR/debug.log & +echo "azurite version = $(azurite --version)" \ No newline at end of file diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt index 6d0f8dc3bb53b..ad96488f44b92 100644 --- a/cpp/src/arrow/filesystem/CMakeLists.txt +++ b/cpp/src/arrow/filesystem/CMakeLists.txt @@ -38,7 +38,12 @@ if(ARROW_GCS) endif() if(ARROW_AZURE) - add_arrow_test(azurefs_test EXTRA_LABELS filesystem) + add_arrow_test(azurefs_test + EXTRA_LABELS + filesystem + EXTRA_LINK_LIBS + Boost::filesystem + Boost::system) endif() if(ARROW_S3) diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 751192ffa9276..a456d59aa32c6 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -17,8 +17,11 @@ #include "arrow/filesystem/azurefs.h" +#include +#include #include #include +#include #include #include @@ -26,6 +29,7 @@ #include "arrow/testing/future_util.h" #include "arrow/testing/gtest_util.h" #include "arrow/testing/util.h" +#include "arrow/util/io_util.h" #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" #include "arrow/util/uri.h" @@ -37,6 +41,61 @@ using internal::Uri; namespace fs { namespace internal { +namespace bp = boost::process; + +using ::arrow::internal::TemporaryDir; +using ::testing::IsEmpty; +using ::testing::NotNull; + +class AzuriteEmulator : public ::testing::Environment { + public: + AzuriteEmulator() { + account_name_ = "devstoreaccount1"; + account_key_ = + "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/" + "KBHBeksoGMGw=="; + auto exe_path = bp::search_path("azurite"); + if (exe_path.empty()) { + auto error = std::string("Could not find Azurite emulator."); + error_ = std::move(error); + return; + } + auto temp_dir_ = TemporaryDir::Make("azurefs-test-").ValueOrDie(); + server_process_ = bp::child(boost::this_process::environment(), exe_path, "--silent", + "--location", temp_dir_->path().ToString(), "--debug", + temp_dir_->path().ToString() + "/debug.log"); + if (!(server_process_.valid() && server_process_.running())) { + auto error = "Could not start Azurite emulator."; + server_process_.terminate(); + server_process_.wait(); + error_ = std::move(error); + } + } + + ~AzuriteEmulator() override { + server_process_.terminate(); + server_process_.wait(); + } + + const std::string& account_name() const { return account_name_; } + const std::string& account_key() const { return account_key_; } + const std::string& error() const { return error_; } + + private: + std::string account_name_; + std::string account_key_; + bp::child server_process_; + std::string error_; + std::unique_ptr temp_dir_; +}; + +AzuriteEmulator* TestAzure() { + static auto* const environment = [] { return new AzuriteEmulator; }(); + return environment; +} + +auto* testazure_env = ::testing::AddGlobalTestEnvironment(TestAzure()); + class TestAzureFileSystem : public ::testing::Test { public: std::shared_ptr fs_; @@ -44,12 +103,12 @@ class TestAzureFileSystem : public ::testing::Test { AzureOptions options_; void MakeFileSystem() { - const std::string& account_name = "devstoreaccount1"; - const std::string& account_key = - "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/" - "KBHBeksoGMGw=="; - options_.account_blob_url = "http://127.0.0.1:10000/devstoreaccount1/"; - options_.account_dfs_url = "http://127.0.0.1:10000/devstoreaccount1/"; + const std::string& account_name = TestAzure()->account_name(); + const std::string& account_key = TestAzure()->account_key(); + options_.account_blob_url = + "http://127.0.0.1:10000/" + TestAzure()->account_name() + "/"; + options_.account_dfs_url = + "http://127.0.0.1:10000/" + TestAzure()->account_name() + "/"; options_.isTestEnabled = true; options_.storage_credentials_provider = std::make_shared(account_name, @@ -63,6 +122,9 @@ class TestAzureFileSystem : public ::testing::Test { } void SetUp() override { + ASSERT_THAT(TestAzure(), NotNull()); + ASSERT_THAT(TestAzure()->error(), IsEmpty()); + MakeFileSystem(); auto fileSystemClient = gen2Client_->GetFileSystemClient("container"); fileSystemClient.CreateIfNotExists(); From 8248c48adab5e2861be551445a8fbb55153e3191 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 14 Jul 2022 22:22:54 +0530 Subject: [PATCH 27/34] Fixed naming in azurefs_test.cc --- .github/workflows/cpp.yml | 2 +- ci/docker/ubuntu-20.04-cpp.dockerfile | 3 +- ci/docker/ubuntu-22.04-cpp.dockerfile | 3 +- ci/scripts/install_azurite.sh | 3 - cpp/Brewfile | 1 + cpp/src/arrow/filesystem/azurefs.cc | 137 +++++++-- cpp/src/arrow/filesystem/azurefs.h | 2 +- cpp/src/arrow/filesystem/azurefs_test.cc | 357 +++++++++++------------ 8 files changed, 285 insertions(+), 223 deletions(-) diff --git a/.github/workflows/cpp.yml b/.github/workflows/cpp.yml index f2b2da6f158ab..ecd7e89b4f4b6 100644 --- a/.github/workflows/cpp.yml +++ b/.github/workflows/cpp.yml @@ -163,7 +163,7 @@ jobs: run: ci/scripts/install_gcs_testbench.sh default - name: Install Azurite Storage Emulator shell: bash - run: ci/scripts/install_azurite.sh $(pwd) + run: ci/scripts/install_azurite.sh - name: Setup ccache run: | ci/scripts/ccache_setup.sh diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index 2daa2c3222a2d..af288fa9be123 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -94,6 +94,7 @@ RUN apt-get update -y -q && \ make \ ninja-build \ nlohmann-json3-dev \ + npm \ pkg-config \ protobuf-compiler \ python3-pip \ @@ -113,7 +114,7 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN /arrow/ci/scripts/install_azurite.sh /usr/local +RUN /arrow/ci/scripts/install_azurite.sh COPY ci/scripts/install_ceph.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_ceph.sh diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index e105a8f47006f..19425805c7818 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -93,6 +93,7 @@ RUN apt-get update -y -q && \ make \ ninja-build \ nlohmann-json3-dev \ + npm \ pkg-config \ protobuf-compiler \ protobuf-compiler-grpc \ @@ -144,7 +145,7 @@ COPY ci/scripts/install_gcs_testbench.sh /arrow/ci/scripts/ RUN /arrow/ci/scripts/install_gcs_testbench.sh default COPY ci/scripts/install_azurite.sh /arrow/ci/scripts/ -RUN /arrow/ci/scripts/install_azurite.sh /usr/local +RUN /arrow/ci/scripts/install_azurite.sh # Prioritize system packages and local installation # The following dependencies will be downloaded due to missing/invalid packages diff --git a/ci/scripts/install_azurite.sh b/ci/scripts/install_azurite.sh index a76cd8b01765d..2e7008360fdc3 100755 --- a/ci/scripts/install_azurite.sh +++ b/ci/scripts/install_azurite.sh @@ -21,7 +21,6 @@ set -e case "$(uname)" in Darwin) - brew install node npm install -g azurite which azurite ;; @@ -30,8 +29,6 @@ case "$(uname)" in npm install -g azurite ;; Linux) - apt-get update - apt-get -y install npm npm install -g azurite which azurite ;; diff --git a/cpp/Brewfile b/cpp/Brewfile index 52b86e4527a7e..179d105be0773 100644 --- a/cpp/Brewfile +++ b/cpp/Brewfile @@ -31,6 +31,7 @@ brew "llvm@12" brew "lz4" brew "minio" brew "ninja" +brew "node" brew "numpy" brew "openssl@1.1" brew "protobuf" diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index e1ed328cc2d9a..a4e7524c9a654 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -115,8 +115,13 @@ void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, const std::string& account_key) { - account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; - account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + if (this->is_azurite) { + account_blob_url = "http://127.0.0.1:10000/" + account_name + "/"; + account_dfs_url = "http://127.0.0.1:10000/" + account_name + "/"; + } else { + account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; + account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; + } storage_credentials_provider = std::make_shared(account_name, account_key); @@ -541,10 +546,14 @@ class ObjectOutputStream final : public io::OutputStream { ObjectOutputStream( std::shared_ptr pathClient, std::shared_ptr fileClient, + std::shared_ptr blobClient, + const bool& isHierarchicalNamespaceEnabled, const io::IOContext& io_context, const AzurePath& path, const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), + blobClient_(std::move(blobClient)), + isHierarchicalNamespaceEnabled_(isHierarchicalNamespaceEnabled), io_context_(io_context), path_(path), metadata_(metadata) {} @@ -566,15 +575,13 @@ class ObjectOutputStream final : public io::OutputStream { if (properties.Value.IsDirectory) { return Status::IOError("Invalid file path given"); } - content_length_ = properties.Value.FileSize; - DCHECK_GE(content_length_, 0); } catch (std::exception const& e) { // new file - std::string s = ""; - fileClient_->UploadFrom( - const_cast(reinterpret_cast(&s[0])), s.size()); - content_length_ = 0; } + std::string s = ""; + fileClient_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + content_length_ = 0; return Status::OK(); } @@ -622,12 +629,25 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto result = - fileClient_ - ->UploadFrom(const_cast(reinterpret_cast(data)), - nbytes) - .Value; - pos_ += nbytes; + if (isHierarchicalNamespaceEnabled_) { + auto bufferStream = std::make_unique( + Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes)); + auto result = fileClient_->Append(*bufferStream, pos_); + fileClient_->Flush(pos_); + } else { + auto append_data = static_cast((void*)data); + auto res = blobClient_->GetBlockList().Value; + std::string text = std::to_string(rand()); + const std::string blockId = Azure::Core::Convert::Base64Encode(std::vector(text.begin(), text.end())); + auto blockContent = Azure::Core::IO::MemoryBodyStream(append_data, strlen(reinterpret_cast(append_data))); + blobClient_->StageBlock(blockId, blockContent); + std::vector blockIds; + for (auto r: res.CommittedBlocks) { + blockIds.push_back(r.Name); + } + blockIds.push_back(blockId); + blobClient_->CommitBlockList(blockIds); + } return Status::OK(); } @@ -642,6 +662,8 @@ class ObjectOutputStream final : public io::OutputStream { protected: std::shared_ptr pathClient_; std::shared_ptr fileClient_; + std::shared_ptr blobClient_; + const bool isHierarchicalNamespaceEnabled_; const io::IOContext io_context_; const AzurePath path_; @@ -656,10 +678,14 @@ class ObjectAppendStream final : public io::OutputStream { ObjectAppendStream( std::shared_ptr pathClient, std::shared_ptr fileClient, + std::shared_ptr blobClient, + const bool& isHierarchicalNamespaceEnabled, const io::IOContext& io_context, const AzurePath& path, const std::shared_ptr& metadata) : pathClient_(std::move(pathClient)), fileClient_(std::move(fileClient)), + blobClient_(std::move(blobClient)), + isHierarchicalNamespaceEnabled_(isHierarchicalNamespaceEnabled), io_context_(io_context), path_(path), metadata_(metadata) {} @@ -682,11 +708,14 @@ class ObjectAppendStream final : public io::OutputStream { return Status::IOError("Invalid file path given"); } content_length_ = properties.Value.FileSize; - DCHECK_GE(content_length_, 0); - return Status::OK(); } catch (std::exception const& e) { - return Status::IOError("Invalid file path given"); + // new file + std::string s = ""; + fileClient_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + content_length_ = 0; } + return Status::OK(); } Status Abort() override { @@ -733,9 +762,25 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - auto content = Azure::Core::IO::MemoryBodyStream( - const_cast(reinterpret_cast(data)), nbytes); - auto result = fileClient_->Append(content, 0); + if (isHierarchicalNamespaceEnabled_) { + auto bufferStream = std::make_unique( + Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes)); + auto result = fileClient_->Append(*bufferStream, pos_); + fileClient_->Flush(pos_); + } else { + auto append_data = static_cast((void*)data); + auto res = blobClient_->GetBlockList().Value; + std::string text = std::to_string(rand()); + const std::string blockId = Azure::Core::Convert::Base64Encode(std::vector(text.begin(), text.end())); + auto blockContent = Azure::Core::IO::MemoryBodyStream(append_data, strlen(reinterpret_cast(append_data))); + blobClient_->StageBlock(blockId, blockContent); + std::vector blockIds; + for (auto r: res.CommittedBlocks) { + blockIds.push_back(r.Name); + } + blockIds.push_back(blockId); + blobClient_->CommitBlockList(blockIds); + } return Status::OK(); } @@ -750,6 +795,8 @@ class ObjectAppendStream final : public io::OutputStream { protected: std::shared_ptr pathClient_; std::shared_ptr fileClient_; + std::shared_ptr blobClient_; + const bool isHierarchicalNamespaceEnabled_; const io::IOContext io_context_; const AzurePath path_; @@ -807,7 +854,7 @@ class AzureBlobFileSystem::Impl blob_endpoint_url = options_.account_blob_url; RETURN_NOT_OK(InitServiceClient(gen1Client_, options_, blob_endpoint_url)); RETURN_NOT_OK(InitServiceClient(gen2Client_, options_, dfs_endpoint_url)); - if (options_.isTestEnabled) { + if (options_.is_azurite) { // gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled throws error // in azurite isHierarchicalNamespaceEnabled = false; @@ -1329,6 +1376,11 @@ class AzureBlobFileSystem::Impl fileClient_, options_, endpoint_url + path.full_path, path.container, path.path_to_file)); + std::shared_ptr blobClient_; + RETURN_NOT_OK(InitPathClient( + blobClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file)); + if (path.has_parent()) { AzurePath parent_path = path.parent(); if (parent_path.path_to_file.empty()) { @@ -1344,7 +1396,7 @@ class AzureBlobFileSystem::Impl } } } - auto ptr = std::make_shared(pathClient_, fileClient_, + auto ptr = std::make_shared(pathClient_, fileClient_, blobClient_, isHierarchicalNamespaceEnabled, fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; @@ -1355,23 +1407,52 @@ class AzureBlobFileSystem::Impl AzureBlobFileSystem* fs) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); + if (path.empty() || path.path_to_file.empty()) { + return Status::IOError("Invalid path provided"); + } + std::string endpoint_url = dfs_endpoint_url; if (!isHierarchicalNamespaceEnabled) { if (path.path_to_file_parts.size() > 1) { return Status::IOError( - "Invalid path provided, " - "hierarchical namespace not enabled"); + "Invalid path provided," + " hierarchical namespace not enabled"); } + endpoint_url = blob_endpoint_url; + } + if (DirExists(dfs_endpoint_url + path.full_path).ValueOrDie()) { + return Status::IOError("Invalid path provided"); } - std::shared_ptr pathClient_; RETURN_NOT_OK(InitPathClient( - pathClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file)); + pathClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file)); std::shared_ptr fileClient_; RETURN_NOT_OK(InitPathClient( - fileClient_, options_, dfs_endpoint_url + s, path.container, path.path_to_file)); + fileClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file)); + + std::shared_ptr blobClient_; + RETURN_NOT_OK(InitPathClient( + blobClient_, options_, endpoint_url + path.full_path, path.container, + path.path_to_file)); - auto ptr = std::make_shared(pathClient_, fileClient_, + if (path.has_parent()) { + AzurePath parent_path = path.parent(); + if (parent_path.path_to_file.empty()) { + if (!ContainerExists(parent_path.container).ValueOrDie()) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } else { + auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); + if (!(exists.ValueOrDie())) { + return Status::IOError("Cannot write to file '", path.full_path, + "': parent directory does not exist"); + } + } + } + auto ptr = std::make_shared(pathClient_, fileClient_, blobClient_, isHierarchicalNamespaceEnabled, fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; diff --git a/cpp/src/arrow/filesystem/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h index 4b4ee7f21715f..b77d9d1e05ab1 100644 --- a/cpp/src/arrow/filesystem/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -61,7 +61,7 @@ struct ARROW_EXPORT AzureOptions { std::string scheme; std::string account_dfs_url; std::string account_blob_url; - bool isTestEnabled = false; + bool is_azurite = false; AzureCredentialsKind credentials_kind = AzureCredentialsKind::Anonymous; std::string sas_token; diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index a456d59aa32c6..88bdcea8160a5 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -47,9 +47,9 @@ using ::arrow::internal::TemporaryDir; using ::testing::IsEmpty; using ::testing::NotNull; -class AzuriteEmulator : public ::testing::Environment { +class AzuriteEnv : public ::testing::Environment { public: - AzuriteEmulator() { + AzuriteEnv() { account_name_ = "devstoreaccount1"; account_key_ = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/" @@ -57,7 +57,7 @@ class AzuriteEmulator : public ::testing::Environment { auto exe_path = bp::search_path("azurite"); if (exe_path.empty()) { auto error = std::string("Could not find Azurite emulator."); - error_ = std::move(error); + status_ = Status::Invalid(error); return; } auto temp_dir_ = TemporaryDir::Make("azurefs-test-").ValueOrDie(); @@ -68,120 +68,157 @@ class AzuriteEmulator : public ::testing::Environment { auto error = "Could not start Azurite emulator."; server_process_.terminate(); server_process_.wait(); - error_ = std::move(error); + status_ = Status::Invalid(error); + return; } + status_ = Status::OK(); } - ~AzuriteEmulator() override { + ~AzuriteEnv() override { server_process_.terminate(); server_process_.wait(); } const std::string& account_name() const { return account_name_; } const std::string& account_key() const { return account_key_; } - const std::string& error() const { return error_; } + const Status status() const { return status_; } private: std::string account_name_; std::string account_key_; bp::child server_process_; - std::string error_; + Status status_; std::unique_ptr temp_dir_; }; -AzuriteEmulator* TestAzure() { - static auto* const environment = [] { return new AzuriteEmulator; }(); - return environment; -} +auto* azurite_env = ::testing::AddGlobalTestEnvironment(new AzuriteEnv); -auto* testazure_env = ::testing::AddGlobalTestEnvironment(TestAzure()); +AzuriteEnv* GetAzuriteEnv() { + return ::arrow::internal::checked_cast(azurite_env); +} class TestAzureFileSystem : public ::testing::Test { public: std::shared_ptr fs_; - std::shared_ptr gen2Client_; + std::shared_ptr gen2_client_; AzureOptions options_; void MakeFileSystem() { - const std::string& account_name = TestAzure()->account_name(); - const std::string& account_key = TestAzure()->account_key(); - options_.account_blob_url = - "http://127.0.0.1:10000/" + TestAzure()->account_name() + "/"; - options_.account_dfs_url = - "http://127.0.0.1:10000/" + TestAzure()->account_name() + "/"; - options_.isTestEnabled = true; - options_.storage_credentials_provider = - std::make_shared(account_name, - account_key); - options_.credentials_kind = AzureCredentialsKind::StorageCredentials; - gen2Client_ = + const std::string& account_name = GetAzuriteEnv()->account_name(); + const std::string& account_key = GetAzuriteEnv()->account_key(); + options_.is_azurite = true; + options_.ConfigureAccountKeyCredentials(account_name, account_key); + gen2_client_ = std::make_shared( options_.account_dfs_url, options_.storage_credentials_provider); - auto result = AzureBlobFileSystem::Make(options_); - fs_ = *result; + ASSERT_OK_AND_ASSIGN(fs_, AzureBlobFileSystem::Make(options_)); } void SetUp() override { - ASSERT_THAT(TestAzure(), NotNull()); - ASSERT_THAT(TestAzure()->error(), IsEmpty()); + ASSERT_THAT(GetAzuriteEnv(), NotNull()); + ASSERT_THAT(GetAzuriteEnv()->status(), Status::OK()); MakeFileSystem(); - auto fileSystemClient = gen2Client_->GetFileSystemClient("container"); - fileSystemClient.CreateIfNotExists(); - fileSystemClient = gen2Client_->GetFileSystemClient("empty-container"); - fileSystemClient.CreateIfNotExists(); - auto fileClient = + auto file_system_client = gen2_client_->GetFileSystemClient("container"); + file_system_client.CreateIfNotExists(); + file_system_client = gen2_client_->GetFileSystemClient("empty-container"); + file_system_client.CreateIfNotExists(); + auto file_client = std::make_shared( options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); std::string s = "some data"; - fileClient->UploadFrom(const_cast(reinterpret_cast(&s[0])), + file_client->UploadFrom(const_cast(reinterpret_cast(s.data())), s.size()); } + void TearDown() override { - auto containers = gen2Client_->ListFileSystems(); - for (auto c : containers.FileSystems) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(c.Name); - fileSystemClient.DeleteIfExists(); + auto containers = gen2_client_->ListFileSystems(); + for (auto container : containers.FileSystems) { + auto file_system_client = gen2_client_->GetFileSystemClient(container.Name); + file_system_client.DeleteIfExists(); } } + void AssertObjectContents( Azure::Storage::Files::DataLake::DataLakeServiceClient* client, const std::string& container, const std::string& path_to_file, const std::string& expected) { - auto pathClient_ = + auto path_client = std::make_shared( client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); - auto size = pathClient_->GetProperties().Value.FileSize; + auto size = path_client->GetProperties().Value.FileSize; if (size == 0) { + ASSERT_EQ(expected, ""); return; } - auto buf = AllocateResizableBuffer(size, fs_->io_context().pool()); - Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; + auto buf = AllocateBuffer(size, fs_->io_context().pool()); + Azure::Storage::Blobs::DownloadBlobToOptions download_options; Azure::Core::Http::HttpRange range; range.Offset = 0; range.Length = size; - downloadOptions.Range = Azure::Nullable(range); - auto fileClient_ = + download_options.Range = Azure::Nullable(range); + auto file_client = std::make_shared( client->GetUrl() + container + "/" + path_to_file, options_.storage_credentials_provider); - auto result = fileClient_ + auto result = file_client ->DownloadTo(reinterpret_cast(buf->get()->mutable_data()), - size, downloadOptions) + size, download_options) .Value; - buf->get()->Equals( - Buffer(const_cast(reinterpret_cast(&expected[0])), - expected.size())); + auto buf_data = std::move(buf->get()); + auto expected_data = std::make_shared(reinterpret_cast(expected.data()), expected.size()); + AssertBufferEqual(*buf_data, *expected_data); } }; +TEST_F(TestAzureFileSystem, FromUri) { + Uri uri; + + // Public container + ASSERT_OK(uri.Parse("https://testcontainer.dfs.core.windows.net/")); + ASSERT_OK_AND_ASSIGN(auto options, AzureOptions::FromUri(uri)); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Anonymous); + ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); + + // Sas Token + ASSERT_OK(uri.Parse("https://testcontainer.blob.core.windows.net/?dummy_sas_token")); + ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); + ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); + ASSERT_EQ(options.sas_token, "?dummy_sas_token"); +} + +TEST_F(TestAzureFileSystem, FromAccountKey) { + AzureOptions options = AzureOptions::FromAccountKey(GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::StorageCredentials); + ASSERT_NE(options.storage_credentials_provider, nullptr); +} + +TEST_F(TestAzureFileSystem, FromConnectionString) { + AzureOptions options = AzureOptions::FromConnectionString("DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;"); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ConnectionString); + ASSERT_NE(options.connection_string, ""); +} + +TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { + AzureOptions options = AzureOptions::FromServicePrincipleCredential("dummy_account_name", "dummy_tenant_id", "dummy_client_id", "dummy_client_secret"); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ServicePrincipleCredentials); + ASSERT_NE(options.service_principle_credentials_provider, nullptr); +} + +TEST_F(TestAzureFileSystem, FromSas) { + AzureOptions options = AzureOptions::FromSas("https://testcontainer.blob.core.windows.net/?dummy_sas_token"); + ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); + ASSERT_NE(options.sas_token, ""); +} + TEST_F(TestAzureFileSystem, CreateDir) { // New container - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); - ASSERT_OK(fs_->CreateDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::Directory); + AssertFileInfo(fs_.get(), "new-container", FileType::NotFound); + ASSERT_OK(fs_->CreateDir("new-container")); + AssertFileInfo(fs_.get(), "new-container", FileType::Directory); // Existing container ASSERT_OK(fs_->CreateDir("container")); @@ -192,10 +229,10 @@ TEST_F(TestAzureFileSystem, CreateDir) { // Existing "file", should fail ASSERT_RAISES(IOError, fs_->CreateDir("container/somefile")); - // directory, false + // recursive, false ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", false)); - // directory, true + // recursive, true ASSERT_RAISES(IOError, fs_->CreateDir("container/newdir/newsub/newsubsub", true)); } @@ -204,50 +241,38 @@ TEST_F(TestAzureFileSystem, DeleteDir) { ASSERT_OK(fs_->DeleteDir("container")); AssertFileInfo(fs_.get(), "container", FileType::NotFound); - // Nonexistent Container - ASSERT_OK(fs_->DeleteDir("container3")); - AssertFileInfo(fs_.get(), "container3", FileType::NotFound); + // Nonexistent-Container + ASSERT_OK(fs_->DeleteDir("nonexistent-container")); + AssertFileInfo(fs_.get(), "nonexistent-container", FileType::NotFound); // root ASSERT_RAISES(NotImplemented, fs_->DeleteDir("")); - // C/F + // Container/File ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile")); - // C/NF - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somefile19")); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); + // Container/Nonexistent-File + ASSERT_RAISES(IOError, fs_->DeleteDir("container/nonexistent-file")); } TEST_F(TestAzureFileSystem, DeleteFile) { // Container ASSERT_RAISES(IOError, fs_->DeleteFile("container")); - // Nonexistent Container - ASSERT_RAISES(IOError, fs_->DeleteFile("container5")); + // Nonexistent-Container + ASSERT_RAISES(IOError, fs_->DeleteFile("nonexistent-container")); // root ASSERT_RAISES(IOError, fs_->DeleteFile("")); - // C/F + // Container/File ASSERT_OK(fs_->DeleteFile("container/somefile")); - // C/NF + // Container/Nonexistent-File ASSERT_RAISES(IOError, fs_->DeleteFile("container/somefile")); - // C/D/D + // Container/Directory/Directory ASSERT_RAISES(IOError, fs_->DeleteFile("container/somedir/subdir")); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDir("container3/somedir")); } TEST_F(TestAzureFileSystem, GetFileInfo) { @@ -257,29 +282,23 @@ TEST_F(TestAzureFileSystem, GetFileInfo) { AssertFileInfo(fs_.get(), "", FileType::Directory); - auto res = fs_->OpenOutputStream("container/base.txt"); - ASSERT_OK(res->get()->Write("Base data")); + ASSERT_OK_AND_ASSIGN(auto res, fs_->OpenOutputStream("container/base.txt")); + ASSERT_OK(res->Write("Base data")); // "Files" AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); - AssertFileInfo(fs_.get(), "container/base1.txt", FileType::NotFound); - - // "Directories" - AssertFileInfo(fs_.get(), "container/non-existentdir/subdir", FileType::NotFound); - AssertFileInfo(fs_.get(), "nonexistent-container/somedir/subdir/subfile", - FileType::NotFound); + AssertFileInfo(fs_.get(), "container/nonexistent-file.txt", FileType::NotFound); } TEST_F(TestAzureFileSystem, GetFileInfoSelector) { FileSelector select; - std::vector infos; // Non-empty container select.base_dir = "container"; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); + ASSERT_OK_AND_ASSIGN(auto infos, fs_->GetFileInfo(select)); ASSERT_EQ(infos.size(), 1); - // Nonexistent container + // Nonexistent-Container select.base_dir = "nonexistent-container"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); select.allow_not_found = true; @@ -292,25 +311,9 @@ TEST_F(TestAzureFileSystem, GetFileInfoSelector) { ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); ASSERT_EQ(infos.size(), 2); - // C/F + // Container/File select.base_dir = "container/somefile"; ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - - // C/ND/D - select.base_dir = "container/non-existentdir/non-existentsubdir"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; - - // NC/D - select.base_dir = "nonexistent-container/non-existentdir"; - ASSERT_RAISES(IOError, fs_->GetFileInfo(select)); - select.allow_not_found = true; - ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select)); - ASSERT_EQ(infos.size(), 0); - select.allow_not_found = false; } TEST_F(TestAzureFileSystem, Move) { @@ -318,43 +321,25 @@ TEST_F(TestAzureFileSystem, Move) { ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir", - "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, - fs_->Move("container/emptydir", "nonexistent-container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->Move("container/emptydir23", "container/base.txt")); - auto res = fs_->OpenOutputStream("container/somefile"); - ASSERT_OK(res->get()->Write("Changed the data")); + ASSERT_RAISES(IOError, fs_->Move("container/nonexistent-directory", "container/base.txt")); + ASSERT_OK_AND_ASSIGN(auto res, fs_->OpenOutputStream("container/somefile")); + ASSERT_OK(res->Write("Changed the data")); ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/base.txt", - "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, - fs_->Move("container/base.txt", "nonexistent-container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->Move("container/base2.txt", "container/non-existentdir")); + ASSERT_RAISES(IOError, fs_->Move("container/nonexistent-file.txt", "container/non-existentdir")); } TEST_F(TestAzureFileSystem, CopyFile) { ASSERT_RAISES(IOError, fs_->CopyFile("container", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir", "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir", "container/newfile")); - ASSERT_RAISES(IOError, - fs_->CopyFile("container/somedir22/subdir", "container/newfile")); - ASSERT_RAISES(IOError, fs_->CopyFile("nonexistent-container/somedir/subdir", - "container/newfile")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container3435")); + ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "nonexistent-container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "")); ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", - "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base.txt", - "nonexistent-container/non-existentdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/base2t.txt", - "nonexistent-container/non-existentdir")); ASSERT_OK(fs_->CopyFile("container/somefile", "container/base.txt")); - ASSERT_OK(fs_->CopyFile("container/base.txt", "container/somefile3")); + ASSERT_OK(fs_->CopyFile("container/base.txt", "container/nonexistent-file")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "nonexistent-container")); @@ -365,28 +350,19 @@ TEST_F(TestAzureFileSystem, CopyFile) { "container/non-existentdir")); ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", "container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "container/non-existentdir/non-existentsubdir")); - ASSERT_RAISES(IOError, fs_->CopyFile("container/somedir/subdir/subfile", - "nonexistent-container/non-existentdir")); } TEST_F(TestAzureFileSystem, OpenInputStream) { - std::shared_ptr stream; - std::shared_ptr buf; - ASSERT_RAISES(IOError, fs_->OpenInputStream("container")); ASSERT_RAISES(IOError, fs_->OpenInputStream("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenInputStream("")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container/non-existentdir")); ASSERT_RAISES(IOError, fs_->OpenInputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("container/non-existentdir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputStream("nonexistent-container/somedir")); // "Files" - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenInputStream("container/somefile")); - ASSERT_OK_AND_ASSIGN(buf, stream->Read(2)); + ASSERT_OK_AND_ASSIGN(auto stream, fs_->OpenInputStream("container/somefile")); + ASSERT_OK_AND_ASSIGN(auto buf, stream->Read(2)); AssertBufferEqual(*buf, "so"); ASSERT_OK_AND_ASSIGN(buf, stream->Read(5)); AssertBufferEqual(*buf, "me da"); @@ -397,22 +373,17 @@ TEST_F(TestAzureFileSystem, OpenInputStream) { } TEST_F(TestAzureFileSystem, OpenInputFile) { - std::shared_ptr file; - std::shared_ptr buf; - ASSERT_RAISES(IOError, fs_->OpenInputFile("container")); ASSERT_RAISES(IOError, fs_->OpenInputFile("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenInputFile("")); ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir")); ASSERT_RAISES(IOError, fs_->OpenInputFile("container/non-existentdir")); ASSERT_RAISES(IOError, fs_->OpenInputFile("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("container/non-existentdir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenInputFile("nonexistent-container/somedir")); // "Files" - ASSERT_OK_AND_ASSIGN(file, fs_->OpenInputFile("container/somefile")); + ASSERT_OK_AND_ASSIGN(auto file, fs_->OpenInputFile("container/somefile")); ASSERT_OK_AND_EQ(9, file->GetSize()); - ASSERT_OK_AND_ASSIGN(buf, file->Read(4)); + ASSERT_OK_AND_ASSIGN(auto buf, file->Read(4)); AssertBufferEqual(*buf, "some"); ASSERT_OK_AND_EQ(9, file->GetSize()); ASSERT_OK_AND_EQ(4, file->Tell()); @@ -425,7 +396,7 @@ TEST_F(TestAzureFileSystem, OpenInputFile) { ASSERT_OK_AND_ASSIGN(buf, file->ReadAt(9, 20)); AssertBufferEqual(*buf, ""); - char result[10]; + char result[20]; ASSERT_OK_AND_EQ(5, file->ReadAt(2, 5, &result)); ASSERT_OK_AND_EQ(4, file->ReadAt(5, 20, &result)); ASSERT_OK_AND_EQ(0, file->ReadAt(9, 0, &result)); @@ -444,25 +415,21 @@ TEST_F(TestAzureFileSystem, OpenInputFile) { } TEST_F(TestAzureFileSystem, OpenOutputStream) { - std::shared_ptr stream; - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container")); ASSERT_RAISES(IOError, fs_->OpenOutputStream("nonexistent-container")); ASSERT_RAISES(IOError, fs_->OpenOutputStream("")); ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/somedir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("container/non-existentdir/subdir")); - ASSERT_RAISES(IOError, fs_->OpenOutputStream("nonexistent-container/somedir")); // Create new empty file - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); + ASSERT_OK_AND_ASSIGN(auto stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", ""); // Create new file with 1 small write ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile2")); ASSERT_OK(stream->Write("some data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile2", "some data"); + AssertObjectContents(gen2_client_.get(), "container", "newfile2", "some data"); // Create new file with 3 small writes ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile3")); @@ -470,36 +437,56 @@ TEST_F(TestAzureFileSystem, OpenOutputStream) { ASSERT_OK(stream->Write("")); ASSERT_OK(stream->Write("new data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile3", "some new data"); - - // Create new file with some large writes - std::string s1, s2, s3, s4, s5, expected; - s1 = random_string(6000000, /*seed =*/42); // More than the 5 MB minimum part upload - s2 = "xxx"; - s3 = random_string(6000000, 43); - s4 = "zzz"; - s5 = random_string(600000, 44); - expected = s1 + s2 + s3 + s4 + s5; - ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile4")); - for (auto input : {s1, s2, s3, s4, s5}) { - ASSERT_OK(stream->Write(input)); - // Clobber source contents. This shouldn't reflect in the data written. - input.front() = 'x'; - input.back() = 'x'; - } - ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile4", expected); + AssertObjectContents(gen2_client_.get(), "container", "newfile3", "some new data"); // Overwrite ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Write("overwritten data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", "overwritten data"); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", "overwritten data"); // Overwrite and make empty ASSERT_OK_AND_ASSIGN(stream, fs_->OpenOutputStream("container/newfile1")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2Client_.get(), "container", "newfile1", ""); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", ""); +} + +TEST_F(TestAzureFileSystem, OpenAppendStream) { + ASSERT_RAISES(IOError, fs_->OpenAppendStream("container")); + ASSERT_RAISES(IOError, fs_->OpenAppendStream("nonexistent-container")); + ASSERT_RAISES(IOError, fs_->OpenAppendStream("")); + ASSERT_RAISES(IOError, fs_->OpenAppendStream("container/somedir/subdir")); + + // Create new empty file + ASSERT_OK_AND_ASSIGN(auto stream, fs_->OpenAppendStream("container/newfile1")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", ""); + + // Create new file with 1 small write + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenAppendStream("container/newfile2")); + ASSERT_OK(stream->Write("some data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2_client_.get(), "container", "newfile2", "some data"); + + // Create new file with 3 small writes + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenAppendStream("container/newfile3")); + ASSERT_OK(stream->Write("some ")); + ASSERT_OK(stream->Write("")); + ASSERT_OK(stream->Write("new data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2_client_.get(), "container", "newfile3", "some new data"); + + // Append to empty file + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenAppendStream("container/newfile1")); + ASSERT_OK(stream->Write("append data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", "append data"); + + // Append to non-empty file + ASSERT_OK_AND_ASSIGN(stream, fs_->OpenAppendStream("container/newfile1")); + ASSERT_OK(stream->Write(", more data")); + ASSERT_OK(stream->Close()); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", "append data, more data"); } TEST_F(TestAzureFileSystem, DeleteDirContents) { @@ -507,24 +494,18 @@ TEST_F(TestAzureFileSystem, DeleteDirContents) { ASSERT_OK(fs_->DeleteDirContents("container")); AssertFileInfo(fs_.get(), "container", FileType::Directory); - // Nonexistent Container - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3")); + // Nonexistent-Container + ASSERT_RAISES(IOError, fs_->DeleteDirContents("nonexistent-container")); AssertFileInfo(fs_.get(), "container3", FileType::NotFound); // root ASSERT_RAISES(IOError, fs_->DeleteDirContents("")); - // C/F - auto res = fs_->OpenOutputStream("container/somefile"); - ASSERT_OK(res->get()->Write("some data")); + // Container/File + ASSERT_OK_AND_ASSIGN(auto res, fs_->OpenOutputStream("container/somefile")); + ASSERT_OK(res->Write("some data")); ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somefile")); AssertFileInfo(fs_.get(), "container/somefile", FileType::File); - - // C/ND/D - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container/somedir3/base")); - - // NC/D - ASSERT_RAISES(IOError, fs_->DeleteDirContents("container3/somedir")); } } // namespace internal From dcd6e30678e27ed164ae8fe6bdef71c5815537b5 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Tue, 19 Jul 2022 21:44:55 +0530 Subject: [PATCH 28/34] Fixed naming in azurefs.cc --- cpp/src/arrow/filesystem/azurefs.cc | 1228 ++++++++++++---------- cpp/src/arrow/filesystem/azurefs.h | 39 +- cpp/src/arrow/filesystem/azurefs_test.cc | 51 +- 3 files changed, 752 insertions(+), 566 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index a4e7524c9a654..0f983e1df7962 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -19,21 +19,23 @@ #include #include -#include -#include -#include -#include -#include #include #include #include #include #include +#include #include #include #include #include +#include +#include +#include +#include +#include + #include "arrow/util/windows_fixup.h" #include "arrow/buffer.h" @@ -68,53 +70,34 @@ static const char kSep = '/'; AzureOptions::AzureOptions() {} -std::string AzureOptions::GetAccountNameFromConnectionString( - const std::string& connectionString) { - std::map connectionStringMap; - std::string::const_iterator cur = connectionString.begin(); - - while (cur != connectionString.end()) { - auto key_begin = cur; - auto key_end = std::find(cur, connectionString.end(), '='); - std::string key = std::string(key_begin, key_end); - cur = key_end; - if (cur != connectionString.end()) { - ++cur; - } - auto value_begin = cur; - auto value_end = std::find(cur, connectionString.end(), ';'); - std::string value = std::string(value_begin, value_end); - cur = value_end; - if (cur != connectionString.end()) { - ++cur; - } - if (!key.empty() || !value.empty()) { - connectionStringMap[std::move(key)] = std::move(value); - } - } - - auto getWithDefault = [](const std::map& m, - const std::string& key, - const std::string& defaultValue = std::string()) { - auto ite = m.find(key); - return ite == m.end() ? defaultValue : ite->second; - }; - - std::string accountName = getWithDefault(connectionStringMap, "AccountName"); - if (accountName.empty()) { - throw std::runtime_error("Cannot find account name in connection string."); - } - return accountName; +Result AzureOptions::GetAccountNameFromConnectionString( + const std::string& connection_string) { + std::string text = "AccountName="; + auto pos_text = connection_string.find(text); + if (pos_text == std::string::npos) { + return Status::IOError( + "Cannot find account name in Azure Blob Storage connection string: '", + connection_string, "'"); + } + auto pos_colon = connection_string.find(';'); + pos_colon = connection_string.find(';', pos_colon + 1); + if (pos_colon == std::string::npos) { + return Status::IOError("Invalid Azure Blob Storage connection string: '", + connection_string, "' passed"); + } + std::string account_name = connection_string.substr(pos_text + text.size(), pos_colon); + return account_name; } -void AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name) { +Status AzureOptions::ConfigureAnonymousCredentials(const std::string& account_name) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; credentials_kind = AzureCredentialsKind::Anonymous; + return Status::OK(); } -void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, - const std::string& account_key) { +Status AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_name, + const std::string& account_key) { if (this->is_azurite) { account_blob_url = "http://127.0.0.1:10000/" + account_name + "/"; account_dfs_url = "http://127.0.0.1:10000/" + account_name + "/"; @@ -126,18 +109,21 @@ void AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_nam std::make_shared(account_name, account_key); credentials_kind = AzureCredentialsKind::StorageCredentials; + return Status::OK(); } -void AzureOptions::ConfigureConnectionStringCredentials( +Status AzureOptions::ConfigureConnectionStringCredentials( const std::string& connection_string_uri) { - auto account_name = GetAccountNameFromConnectionString(connection_string_uri); + ARROW_ASSIGN_OR_RAISE(auto account_name, + GetAccountNameFromConnectionString(connection_string_uri)); account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; connection_string = connection_string_uri; credentials_kind = AzureCredentialsKind::ConnectionString; + return Status::OK(); } -void AzureOptions::ConfigureServicePrincipleCredentials( +Status AzureOptions::ConfigureServicePrincipleCredentials( const std::string& account_name, const std::string& tenant_id, const std::string& client_id, const std::string& client_secret) { account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; @@ -146,18 +132,17 @@ void AzureOptions::ConfigureServicePrincipleCredentials( std::make_shared(tenant_id, client_id, client_secret); credentials_kind = AzureCredentialsKind::ServicePrincipleCredentials; + return Status::OK(); } -void AzureOptions::ConfigureSasCredentials(const std::string& uri) { - auto src = internal::RemoveTrailingSlash(uri); - auto first_sep = src.find_first_of("?"); - sas_token = std::string(src.substr(first_sep)); - account_blob_url = std::string(src.substr(0, first_sep)); - src = internal::RemoveTrailingSlash(account_blob_url); - first_sep = src.find("blob.core.windows.net"); - account_dfs_url = std::string(src.substr(0, first_sep)) + "dfs" + - std::string(src.substr(first_sep + 4)) + "/"; +Status AzureOptions::ConfigureSasCredentials(const std::string& uri) { + Uri url; + RETURN_NOT_OK(url.Parse(uri)); + sas_token = "?" + url.query_string(); + account_blob_url = url.scheme() + "://" + url.host() + kSep; + account_dfs_url = std::regex_replace(account_blob_url, std::regex(".blob"), ".dfs"); credentials_kind = AzureCredentialsKind::Sas; + return Status::OK(); } bool AzureOptions::Equals(const AzureOptions& other) const { @@ -166,37 +151,38 @@ bool AzureOptions::Equals(const AzureOptions& other) const { credentials_kind == other.credentials_kind); } -AzureOptions AzureOptions::FromAnonymous(const std::string account_name) { +Result AzureOptions::FromAnonymous(const std::string& account_name) { AzureOptions options; - options.ConfigureAnonymousCredentials(account_name); + RETURN_NOT_OK(options.ConfigureAnonymousCredentials(account_name)); return options; } -AzureOptions AzureOptions::FromAccountKey(const std::string& account_name, - const std::string& account_key) { +Result AzureOptions::FromAccountKey(const std::string& account_name, + const std::string& account_key) { AzureOptions options; - options.ConfigureAccountKeyCredentials(account_name, account_key); + RETURN_NOT_OK(options.ConfigureAccountKeyCredentials(account_name, account_key)); return options; } -AzureOptions AzureOptions::FromConnectionString(const std::string& connection_string) { +Result AzureOptions::FromConnectionString( + const std::string& connection_string) { AzureOptions options; - options.ConfigureConnectionStringCredentials(connection_string); + RETURN_NOT_OK(options.ConfigureConnectionStringCredentials(connection_string)); return options; } -AzureOptions AzureOptions::FromServicePrincipleCredential( +Result AzureOptions::FromServicePrincipleCredential( const std::string& account_name, const std::string& tenant_id, const std::string& client_id, const std::string& client_secret) { AzureOptions options; - options.ConfigureServicePrincipleCredentials(account_name, tenant_id, client_id, - client_secret); + RETURN_NOT_OK(options.ConfigureServicePrincipleCredentials(account_name, tenant_id, + client_id, client_secret)); return options; } -AzureOptions AzureOptions::FromSas(const std::string& uri) { +Result AzureOptions::FromSas(const std::string& uri) { AzureOptions options; - options.ConfigureSasCredentials(uri); + RETURN_NOT_OK(options.ConfigureSasCredentials(uri)); return options; } @@ -208,35 +194,49 @@ Result AzureOptions::FromUri(const std::string& uri_string, } Result AzureOptions::FromUri(const Uri& uri, std::string* out_path) { + // uri = + // https://accountName.dfs.core.windows.net/pathToBlob/?sas_token_key=sas_token_value AzureOptions options; - const auto container = uri.host(); - auto path = uri.path(); - if (container.empty()) { - if (!path.empty()) { - return Status::Invalid("Missing container in URI"); - } + // host = accountName.dfs.core.windows.net + const auto host = uri.host(); + // path_to_blob = /pathToBlob/ + const auto path_to_blob = uri.path(); + std::string account_name; + if (host.empty()) { + return Status::IOError("Missing container in Azure Blob Storage URI: '", + uri.ToString(), "'"); + } + auto pos = host.find('.'); + if (pos == std::string::npos) { + return Status::IOError("Missing container in Azure Blob Storage URI: '", + uri.ToString(), "'"); + } + std::string full_path; + // account_name = accountName + account_name = host.substr(0, pos); + if (full_path.empty()) { + full_path = account_name; } else { - if (path.empty()) { - path = container.substr(0, container.find('.')); - } else { - if (path[0] != '/') { - return Status::Invalid("URI should absolute, not relative"); - } - path = container + path; + if (full_path[0] != '/') { + return Status::IOError("Azure Blob Storage URI should be absolute, not relative"); } + // full_path = accountName/pathToBlob/ + full_path = account_name + path_to_blob; } if (out_path != nullptr) { - *out_path = std::string(internal::RemoveTrailingSlash(path)); + *out_path = std::string(internal::RemoveTrailingSlash(full_path)); } - + // scheme = https options.scheme = uri.scheme(); - AZURE_ASSERT(container.find('.') != std::string::npos); - std::string accountName = container.substr(0, container.find('.')); + // query_string = sas_token_key=sas_token_value const auto query_string = uri.query_string(); if (!query_string.empty()) { - options.ConfigureSasCredentials(uri.scheme() + "://" + path + "?" + query_string); + // Accepted Uri = + // https://accountName.dfs.core.windows.net/pathToBlob/?sas_token_key=sas_token_value + RETURN_NOT_OK(options.ConfigureSasCredentials(uri.scheme() + "://" + host + + path_to_blob + "?" + query_string)); } else { - options.ConfigureAnonymousCredentials(accountName); + RETURN_NOT_OK(options.ConfigureAnonymousCredentials(account_name)); } return options; } @@ -259,15 +259,15 @@ struct AzurePath { // Expected input here => s = synapsemlfs/testdir/testfile.txt auto src = internal::RemoveTrailingSlash(s); if ((src.find("127.0.0.1") != std::string::npos)) { - RETURN_NOT_OK(FromLocalHostString(src)); + RETURN_NOT_OK(FromLocalHostString(&src)); } if (internal::IsLikelyUri(src)) { - RETURN_NOT_OK(ExtractBlobPath(src)); + RETURN_NOT_OK(ExtractBlobPath(&src)); } src = internal::RemoveLeadingSlash(src); auto first_sep = src.find_first_of(kSep); if (first_sep == 0) { - return Status::Invalid("Path cannot start with a separator ('", s, "')"); + return Status::IOError("Path cannot start with a separator ('", s, "')"); } if (first_sep == std::string::npos) { return AzurePath{std::string(src), std::string(src), "", {}}; @@ -281,21 +281,31 @@ struct AzurePath { return path; } - static Status FromLocalHostString(util::string_view& src) { - auto port = src.find("127.0.0.1"); - src = src.substr(port); - auto first_sep = src.find_first_of(kSep); - src = src.substr(first_sep + 1); - auto sec_sep = src.find_first_of(kSep); - src = src.substr(sec_sep + 1); + static Status FromLocalHostString(util::string_view* src) { + // src = http://127.0.0.1:10000/accountName/pathToBlob + auto port = src->find("127.0.0.1"); + // src = 127.0.0.1:10000/accountName/pathToBlob + *src = src->substr(port); + auto first_sep = src->find_first_of(kSep); + if (first_sep == std::string::npos) { + return Status::IOError("Missing account name in Azure Blob Storage URI"); + } + // src = accountName/pathToBlob + *src = src->substr(first_sep + 1); + auto sec_sep = src->find_first_of(kSep); + if (sec_sep == std::string::npos) { + return Status::IOError("Missing container name in Azure Blob Storage URI"); + } + // src = pathToBlob + *src = src->substr(sec_sep + 1); return Status::OK(); } // Removes scheme, host and port from the uri - static Status ExtractBlobPath(util::string_view& s) { + static Status ExtractBlobPath(util::string_view* s) { Uri uri; - RETURN_NOT_OK(uri.Parse(s.to_string())); - s = uri.path(); + RETURN_NOT_OK(uri.Parse(s->to_string())); + *s = uri.path(); return Status::OK(); } @@ -345,8 +355,9 @@ std::shared_ptr GetObjectMetadata(const ObjectResult& re } template -Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, - const std::string url) { +Result> InitServiceClient(const AzureOptions& options, + const std::string& url) { + std::shared_ptr client; if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(url, options.storage_credentials_provider); } else if (options.credentials_kind == @@ -360,13 +371,15 @@ Status InitServiceClient(std::shared_ptr& client, const AzureOptions options, } else { client = std::make_shared(url); } - return Status::OK(); + return client; } template -Status InitPathClient(std::shared_ptr& client, const AzureOptions options, - const std::string path, const std::string container, - const std::string path_to_file) { +Result> InitPathClient(const AzureOptions& options, + const std::string& path, + const std::string& container, + const std::string& path_to_file) { + std::shared_ptr client; if (options.credentials_kind == AzureCredentialsKind::StorageCredentials) { client = std::make_shared(path, options.storage_credentials_provider); } else if (options.credentials_kind == @@ -389,17 +402,17 @@ Status InitPathClient(std::shared_ptr& client, const AzureOptions options, } else { client = std::make_shared(path); } - return Status::OK(); + return client; } class ObjectInputFile final : public io::RandomAccessFile { public: ObjectInputFile( - std::shared_ptr pathClient, - std::shared_ptr fileClient, + std::shared_ptr& path_client, + std::shared_ptr& file_client, const io::IOContext& io_context, const AzurePath& path, int64_t size = kNoSize) - : pathClient_(std::move(pathClient)), - fileClient_(std::move(fileClient)), + : path_client_(std::move(path_client)), + file_client_(std::move(file_client)), io_context_(io_context), path_(path), content_length_(size) {} @@ -410,16 +423,15 @@ class ObjectInputFile final : public io::RandomAccessFile { return Status::OK(); } try { - auto properties = pathClient_->GetProperties(); + auto properties = path_client_->GetProperties(); if (properties.Value.IsDirectory) { - return Status::IOError("Invalid file path given"); + return ::arrow::fs::internal::NotAFile(path_.full_path); } content_length_ = properties.Value.FileSize; - DCHECK_GE(content_length_, 0); metadata_ = GetObjectMetadata(properties.Value.Metadata); return Status::OK(); - } catch (std::exception const& e) { - return Status::IOError("Invalid file path given"); + } catch (const Azure::Storage::StorageException& exception) { + return ::arrow::fs::internal::PathNotFound(path_.full_path); } } @@ -452,8 +464,8 @@ class ObjectInputFile final : public io::RandomAccessFile { } Status Close() override { - pathClient_ = nullptr; - fileClient_ = nullptr; + path_client_ = nullptr; + file_client_ = nullptr; closed_ = true; return Status::OK(); } @@ -488,16 +500,20 @@ class ObjectInputFile final : public io::RandomAccessFile { } // Read the desired range of bytes - Azure::Storage::Blobs::DownloadBlobToOptions downloadOptions; + Azure::Storage::Blobs::DownloadBlobToOptions download_options; Azure::Core::Http::HttpRange range; range.Offset = position; range.Length = nbytes; - downloadOptions.Range = Azure::Nullable(range); - auto result = - fileClient_->DownloadTo(reinterpret_cast(out), nbytes, downloadOptions) - .Value; - AZURE_ASSERT(result.ContentRange.Length.HasValue()); - return result.ContentRange.Length.Value(); + download_options.Range = Azure::Nullable(range); + try { + auto result = + file_client_ + ->DownloadTo(reinterpret_cast(out), nbytes, download_options) + .Value; + return result.ContentRange.Length.Value(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } Result> ReadAt(int64_t position, int64_t nbytes) override { @@ -530,8 +546,8 @@ class ObjectInputFile final : public io::RandomAccessFile { } protected: - std::shared_ptr pathClient_; - std::shared_ptr fileClient_; + std::shared_ptr path_client_; + std::shared_ptr file_client_; const io::IOContext io_context_; AzurePath path_; @@ -544,19 +560,15 @@ class ObjectInputFile final : public io::RandomAccessFile { class ObjectOutputStream final : public io::OutputStream { public: ObjectOutputStream( - std::shared_ptr pathClient, - std::shared_ptr fileClient, - std::shared_ptr blobClient, - const bool& isHierarchicalNamespaceEnabled, - const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) - : pathClient_(std::move(pathClient)), - fileClient_(std::move(fileClient)), - blobClient_(std::move(blobClient)), - isHierarchicalNamespaceEnabled_(isHierarchicalNamespaceEnabled), + std::shared_ptr& file_client, + std::shared_ptr& blob_client, + const bool is_hierarchical_namespace_enabled, const io::IOContext& io_context, + const AzurePath& path, const std::shared_ptr& metadata) + : file_client_(std::move(file_client)), + blob_client_(std::move(blob_client)), + is_hierarchical_namespace_enabled_(is_hierarchical_namespace_enabled), io_context_(io_context), - path_(path), - metadata_(metadata) {} + path_(path) {} ~ObjectOutputStream() override { // For compliance with the rest of the IO stack, Close rather than Abort, @@ -571,17 +583,13 @@ class ObjectOutputStream final : public io::OutputStream { return Status::OK(); } try { - auto properties = pathClient_->GetProperties(); - if (properties.Value.IsDirectory) { - return Status::IOError("Invalid file path given"); - } - } catch (std::exception const& e) { - // new file + std::string s = ""; + file_client_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + content_length_ = 0; + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - std::string s = ""; - fileClient_->UploadFrom( - const_cast(reinterpret_cast(s.data())), s.size()); - content_length_ = 0; return Status::OK(); } @@ -589,8 +597,8 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::OK(); } - pathClient_ = nullptr; - fileClient_ = nullptr; + file_client_ = nullptr; + blob_client_ = nullptr; closed_ = true; return Status::OK(); } @@ -601,8 +609,8 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::OK(); } - pathClient_ = nullptr; - fileClient_ = nullptr; + file_client_ = nullptr; + blob_client_ = nullptr; closed_ = true; return Status::OK(); } @@ -629,24 +637,35 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - if (isHierarchicalNamespaceEnabled_) { - auto bufferStream = std::make_unique( - Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes)); - auto result = fileClient_->Append(*bufferStream, pos_); - fileClient_->Flush(pos_); + if (is_hierarchical_namespace_enabled_) { + try { + auto buffer_stream = std::make_unique( + Azure::Core::IO::MemoryBodyStream( + const_cast(reinterpret_cast(data)), nbytes)); + auto result = file_client_->Append(*buffer_stream, pos_); + file_client_->Flush(pos_); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } else { - auto append_data = static_cast((void*)data); - auto res = blobClient_->GetBlockList().Value; - std::string text = std::to_string(rand()); - const std::string blockId = Azure::Core::Convert::Base64Encode(std::vector(text.begin(), text.end())); - auto blockContent = Azure::Core::IO::MemoryBodyStream(append_data, strlen(reinterpret_cast(append_data))); - blobClient_->StageBlock(blockId, blockContent); - std::vector blockIds; - for (auto r: res.CommittedBlocks) { - blockIds.push_back(r.Name); + try { + auto append_data = static_cast((void*)data); + auto res = blob_client_->GetBlockList().Value; + std::string text = std::to_string(rand()); + const std::string block_id = Azure::Core::Convert::Base64Encode( + std::vector(text.begin(), text.end())); + auto block_content = Azure::Core::IO::MemoryBodyStream( + append_data, strlen(reinterpret_cast(append_data))); + blob_client_->StageBlock(block_id, block_content); + std::vector block_ids; + for (auto block : res.CommittedBlocks) { + block_ids.push_back(block.Name); + } + block_ids.push_back(block_id); + blob_client_->CommitBlockList(block_ids); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - blockIds.push_back(blockId); - blobClient_->CommitBlockList(blockIds); } return Status::OK(); } @@ -655,40 +674,53 @@ class ObjectOutputStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - fileClient_->Flush(content_length_); + if (is_hierarchical_namespace_enabled_) { + try { + file_client_->Flush(content_length_); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else { + try { + auto res = blob_client_->GetBlockList().Value; + std::vector block_ids; + for (auto block : res.UncommittedBlocks) { + block_ids.push_back(block.Name); + } + blob_client_->CommitBlockList(block_ids); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } return Status::OK(); } protected: - std::shared_ptr pathClient_; - std::shared_ptr fileClient_; - std::shared_ptr blobClient_; - const bool isHierarchicalNamespaceEnabled_; + std::shared_ptr file_client_; + std::shared_ptr blob_client_; + const bool is_hierarchical_namespace_enabled_; const io::IOContext io_context_; const AzurePath path_; bool closed_ = true; int64_t pos_ = 0; int64_t content_length_ = kNoSize; - std::shared_ptr metadata_; }; class ObjectAppendStream final : public io::OutputStream { public: ObjectAppendStream( - std::shared_ptr pathClient, - std::shared_ptr fileClient, - std::shared_ptr blobClient, - const bool& isHierarchicalNamespaceEnabled, - const io::IOContext& io_context, const AzurePath& path, - const std::shared_ptr& metadata) - : pathClient_(std::move(pathClient)), - fileClient_(std::move(fileClient)), - blobClient_(std::move(blobClient)), - isHierarchicalNamespaceEnabled_(isHierarchicalNamespaceEnabled), + std::shared_ptr& path_client, + std::shared_ptr& file_client, + std::shared_ptr& blob_client, + const bool is_hierarchical_namespace_enabled, const io::IOContext& io_context, + const AzurePath& path, const std::shared_ptr& metadata) + : path_client_(std::move(path_client)), + file_client_(std::move(file_client)), + blob_client_(std::move(blob_client)), + is_hierarchical_namespace_enabled_(is_hierarchical_namespace_enabled), io_context_(io_context), - path_(path), - metadata_(metadata) {} + path_(path) {} ~ObjectAppendStream() override { // For compliance with the rest of the IO stack, Close rather than Abort, @@ -703,16 +735,20 @@ class ObjectAppendStream final : public io::OutputStream { return Status::OK(); } try { - auto properties = pathClient_->GetProperties(); + auto properties = path_client_->GetProperties(); if (properties.Value.IsDirectory) { - return Status::IOError("Invalid file path given"); + return ::arrow::fs::internal::NotAFile(path_.full_path); } content_length_ = properties.Value.FileSize; - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { // new file std::string s = ""; - fileClient_->UploadFrom( - const_cast(reinterpret_cast(s.data())), s.size()); + try { + file_client_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } content_length_ = 0; } return Status::OK(); @@ -722,8 +758,9 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::OK(); } - pathClient_ = nullptr; - fileClient_ = nullptr; + path_client_ = nullptr; + file_client_ = nullptr; + blob_client_ = nullptr; closed_ = true; return Status::OK(); } @@ -734,8 +771,9 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::OK(); } - pathClient_ = nullptr; - fileClient_ = nullptr; + path_client_ = nullptr; + file_client_ = nullptr; + blob_client_ = nullptr; closed_ = true; return Status::OK(); } @@ -762,24 +800,35 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - if (isHierarchicalNamespaceEnabled_) { - auto bufferStream = std::make_unique( - Azure::Core::IO::MemoryBodyStream(const_cast(reinterpret_cast(data)), nbytes)); - auto result = fileClient_->Append(*bufferStream, pos_); - fileClient_->Flush(pos_); + if (is_hierarchical_namespace_enabled_) { + try { + auto buffer_stream = std::make_unique( + Azure::Core::IO::MemoryBodyStream( + const_cast(reinterpret_cast(data)), nbytes)); + auto result = file_client_->Append(*buffer_stream, pos_); + file_client_->Flush(pos_); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } else { - auto append_data = static_cast((void*)data); - auto res = blobClient_->GetBlockList().Value; - std::string text = std::to_string(rand()); - const std::string blockId = Azure::Core::Convert::Base64Encode(std::vector(text.begin(), text.end())); - auto blockContent = Azure::Core::IO::MemoryBodyStream(append_data, strlen(reinterpret_cast(append_data))); - blobClient_->StageBlock(blockId, blockContent); - std::vector blockIds; - for (auto r: res.CommittedBlocks) { - blockIds.push_back(r.Name); + try { + auto append_data = static_cast((void*)data); + auto res = blob_client_->GetBlockList().Value; + std::string text = std::to_string(rand()); + const std::string block_id = Azure::Core::Convert::Base64Encode( + std::vector(text.begin(), text.end())); + auto block_content = Azure::Core::IO::MemoryBodyStream( + append_data, strlen(reinterpret_cast(append_data))); + blob_client_->StageBlock(block_id, block_content); + std::vector block_ids; + for (auto block : res.CommittedBlocks) { + block_ids.push_back(block.Name); + } + block_ids.push_back(block_id); + blob_client_->CommitBlockList(block_ids); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - blockIds.push_back(blockId); - blobClient_->CommitBlockList(blockIds); } return Status::OK(); } @@ -788,27 +837,43 @@ class ObjectAppendStream final : public io::OutputStream { if (closed_) { return Status::Invalid("Operation on closed stream"); } - fileClient_->Flush(content_length_); + if (is_hierarchical_namespace_enabled_) { + try { + file_client_->Flush(content_length_); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else { + try { + auto res = blob_client_->GetBlockList().Value; + std::vector block_ids; + for (auto block : res.UncommittedBlocks) { + block_ids.push_back(block.Name); + } + blob_client_->CommitBlockList(block_ids); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } return Status::OK(); } protected: - std::shared_ptr pathClient_; - std::shared_ptr fileClient_; - std::shared_ptr blobClient_; - const bool isHierarchicalNamespaceEnabled_; + std::shared_ptr path_client_; + std::shared_ptr file_client_; + std::shared_ptr blob_client_; + const bool is_hierarchical_namespace_enabled_; const io::IOContext io_context_; const AzurePath path_; bool closed_ = true; int64_t pos_ = 0; int64_t content_length_ = kNoSize; - const std::shared_ptr metadata_; }; -TimePoint ToTimePoint(int secs) { - std::chrono::nanoseconds ns_count(static_cast(secs) * 1000000000); - return TimePoint(std::chrono::duration_cast(ns_count)); +TimePoint ToTimePoint(const Azure::DateTime& dt) { + return std::chrono::time_point_cast( + dt.operator std::chrono::system_clock::time_point()); } void FileObjectToInfo( @@ -816,19 +881,15 @@ void FileObjectToInfo( FileInfo* info) { info->set_type(FileType::File); info->set_size(static_cast(properties.FileSize)); - info->set_mtime(ToTimePoint((int)(std::chrono::duration_cast( - properties.LastModified - Azure::DateTime(1970)) - .count()))); + info->set_mtime(ToTimePoint(properties.LastModified)); } -void PathInfoToFileInfo(const std::string path, const FileType type, const int64_t size, +void PathInfoToFileInfo(const std::string& path, const FileType& type, const int64_t size, const Azure::DateTime dt, FileInfo* info) { info->set_type(type); info->set_size(size); info->set_path(path); - info->set_mtime(ToTimePoint( - (int)(std::chrono::duration_cast(dt - Azure::DateTime(1970)) - .count()))); + info->set_mtime(ToTimePoint(dt)); } } // namespace @@ -840,27 +901,37 @@ class AzureBlobFileSystem::Impl : public std::enable_shared_from_this { public: io::IOContext io_context_; - std::shared_ptr gen1Client_; - std::shared_ptr gen2Client_; - std::string dfs_endpoint_url; - std::string blob_endpoint_url; - bool isHierarchicalNamespaceEnabled; + std::shared_ptr gen1_client_; + std::shared_ptr gen2_client_; + std::string dfs_endpoint_url_; + std::string blob_endpoint_url_; + bool is_hierarchical_namespace_enabled_; explicit Impl(AzureOptions options, io::IOContext io_context) : io_context_(io_context), options_(std::move(options)) {} Status Init() { - dfs_endpoint_url = options_.account_dfs_url; - blob_endpoint_url = options_.account_blob_url; - RETURN_NOT_OK(InitServiceClient(gen1Client_, options_, blob_endpoint_url)); - RETURN_NOT_OK(InitServiceClient(gen2Client_, options_, dfs_endpoint_url)); + dfs_endpoint_url_ = options_.account_dfs_url; + blob_endpoint_url_ = options_.account_blob_url; + ARROW_ASSIGN_OR_RAISE(gen1_client_, + InitServiceClient( + options_, blob_endpoint_url_)); + ARROW_ASSIGN_OR_RAISE( + gen2_client_, + InitServiceClient( + options_, dfs_endpoint_url_)); if (options_.is_azurite) { // gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled throws error // in azurite - isHierarchicalNamespaceEnabled = false; + is_hierarchical_namespace_enabled_ = false; } else { - isHierarchicalNamespaceEnabled = - gen1Client_->GetAccountInfo().Value.IsHierarchicalNamespaceEnabled; + try { + auto response = gen1_client_->GetAccountInfo(); + is_hierarchical_namespace_enabled_ = + response.Value.IsHierarchicalNamespaceEnabled; + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } return Status::OK(); } @@ -869,18 +940,22 @@ class AzureBlobFileSystem::Impl // Create a container. Successful if container already exists. Status CreateContainer(const std::string& container) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(container); - fileSystemClient.CreateIfNotExists(); + auto file_system_client = gen2_client_->GetFileSystemClient(container); + try { + file_system_client.CreateIfNotExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } return Status::OK(); } // Tests to see if a container exists Result ContainerExists(const std::string& container) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + auto file_system_client = gen2_client_->GetFileSystemClient(container); try { - auto properties = fileSystemClient.GetProperties(); + auto properties = file_system_client.GetProperties(); return true; - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { return false; } } @@ -888,13 +963,14 @@ class AzureBlobFileSystem::Impl Result DirExists(const std::string& s) { std::string uri = s; ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file)); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( + options_, uri, path.container, path.path_to_file)); try { - auto properties = pathClient_->GetProperties(); + auto properties = path_client->GetProperties(); return properties.Value.IsDirectory; - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { return false; } } @@ -902,106 +978,143 @@ class AzureBlobFileSystem::Impl Result FileExists(const std::string& s) { std::string uri = s; ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(uri)); - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, uri, path.container, path.path_to_file)); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( + options_, uri, path.container, path.path_to_file)); try { - auto properties = pathClient_->GetProperties(); + auto properties = path_client->GetProperties(); return !properties.Value.IsDirectory; - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { return false; } } Status CreateEmptyDir(const std::string& container, const std::vector& path) { - auto directoryClient = - gen2Client_->GetFileSystemClient(container).GetDirectoryClient(path.front()); + if (path.empty()) { + return CreateContainer(container); + } + auto directory_client = + gen2_client_->GetFileSystemClient(container).GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { - directoryClient = directoryClient.GetSubdirectoryClient(*it); + directory_client = directory_client.GetSubdirectoryClient(*it); ++it; } - directoryClient.CreateIfNotExists(); + try { + directory_client.CreateIfNotExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } return Status::OK(); } Status DeleteContainer(const std::string& container) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(container); - fileSystemClient.DeleteIfExists(); + auto file_system_client = gen2_client_->GetFileSystemClient(container); + try { + file_system_client.DeleteIfExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } return Status::OK(); } Status DeleteDir(const std::string& container, const std::vector& path) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(container); - auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + if (path.empty()) { + return DeleteContainer(container); + } + auto file_system_client = gen2_client_->GetFileSystemClient(container); + auto directory_client = file_system_client.GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { - directoryClient = directoryClient.GetSubdirectoryClient(*it); + directory_client = directory_client.GetSubdirectoryClient(*it); ++it; } - if (FileExists(directoryClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete directory, Invalid Directory Path"); + ARROW_ASSIGN_OR_RAISE(auto response, FileExists(directory_client.GetUrl())); + if (response) { + return ::arrow::fs::internal::NotADir(directory_client.GetUrl()); + } + ARROW_ASSIGN_OR_RAISE(response, DirExists(directory_client.GetUrl())); + if (!response) { + return ::arrow::fs::internal::NotADir(directory_client.GetUrl()); } - if (!DirExists(directoryClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete directory, Invalid Directory Path"); + try { + directory_client.DeleteRecursiveIfExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - directoryClient.DeleteRecursiveIfExists(); return Status::OK(); } Status DeleteFile(const std::string& container, const std::vector& path) { if (path.empty()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + return Status::IOError("Cannot delete file, Invalid Azure Blob Storage file path"); } - if (!isHierarchicalNamespaceEnabled) { + if (!is_hierarchical_namespace_enabled_) { if (path.size() > 1) { return Status::IOError( - "Cannot delete File, Invalid File Path," - " hierarchical namespace not enabled"); + "Cannot delete file, Invalid Azure Blob Storage file path," + " hierarchical namespace not enabled in storage account"); } - auto blobClient = - gen1Client_->GetBlobContainerClient(container).GetBlobClient(path.front()); - if (!FileExists(blobClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + auto blob_client = + gen1_client_->GetBlobContainerClient(container).GetBlobClient(path.front()); + ARROW_ASSIGN_OR_RAISE(auto response, FileExists(blob_client.GetUrl())); + if (!response) { + return ::arrow::fs::internal::NotAFile(blob_client.GetUrl()); + } + try { + blob_client.DeleteIfExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - blobClient.DeleteIfExists(); return Status::OK(); } - auto fileSystemClient = gen2Client_->GetFileSystemClient(container); + auto file_system_client = gen2_client_->GetFileSystemClient(container); if (path.size() == 1) { - auto fileClient = fileSystemClient.GetFileClient(path.front()); - if (DirExists(fileClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + auto file_client = file_system_client.GetFileClient(path.front()); + ARROW_ASSIGN_OR_RAISE(auto response, DirExists(file_client.GetUrl())); + if (response) { + return ::arrow::fs::internal::NotAFile(file_client.GetUrl()); } - if (!FileExists(fileClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + ARROW_ASSIGN_OR_RAISE(response, FileExists(file_client.GetUrl())); + if (!response) { + return ::arrow::fs::internal::NotAFile(file_client.GetUrl()); } try { - fileClient.DeleteIfExists(); - } catch (std::exception const& e) { + file_client.DeleteIfExists(); + } catch (const Azure::Storage::StorageException& exception) { // Azurite throws an exception + if (!options_.is_azurite) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } return Status::OK(); } std::string file_name = path.back(); - auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + auto directory_client = file_system_client.GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != (path.end() - 1)) { - directoryClient = directoryClient.GetSubdirectoryClient(*it); + directory_client = directory_client.GetSubdirectoryClient(*it); ++it; } - auto fileClient = directoryClient.GetFileClient(file_name); - if (DirExists(fileClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + auto file_client = directory_client.GetFileClient(file_name); + ARROW_ASSIGN_OR_RAISE(auto response, DirExists(file_client.GetUrl())); + if (response) { + return ::arrow::fs::internal::NotAFile(file_client.GetUrl()); + } + ARROW_ASSIGN_OR_RAISE(response, FileExists(file_client.GetUrl())); + if (!response) { + return ::arrow::fs::internal::NotAFile(file_client.GetUrl()); } - if (!FileExists(fileClient.GetUrl()).ValueOrDie()) { - return Status::IOError("Cannot delete File, Invalid File Path"); + try { + file_client.DeleteIfExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - fileClient.DeleteIfExists(); return Status::OK(); } @@ -1009,70 +1122,78 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); - if (!isHierarchicalNamespaceEnabled) { - return Status::IOError("Cannot move, Hierarchical namespace not enabled"); + if (!is_hierarchical_namespace_enabled_) { + return Status::IOError( + "Cannot perform move operation, Hierarchical namespace not enabled in storage " + "account"); + } + if (src_path.empty() || src_path.path_to_file.empty()) { + return ::arrow::fs::internal::PathNotFound(src_path.full_path); } - if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || - dest_path.path_to_file.empty()) { - return Status::IOError("Invalid path provided"); + if (dest_path.empty() || dest_path.path_to_file.empty()) { + return ::arrow::fs::internal::PathNotFound(dest_path.full_path); } if (src_path == dest_path) { return Status::OK(); } - if (FileExists(dfs_endpoint_url + src_path.full_path).ValueOrDie()) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(src_path.container); + ARROW_ASSIGN_OR_RAISE(auto file_exists, + FileExists(dfs_endpoint_url_ + src_path.full_path)); + ARROW_ASSIGN_OR_RAISE(auto dir_exists, + DirExists(dfs_endpoint_url_ + src_path.full_path)); + if (file_exists) { + auto file_system_client = gen2_client_->GetFileSystemClient(src_path.container); auto path = src_path.path_to_file_parts; if (path.size() == 1) { try { - fileSystemClient.RenameFile(path.front(), dest_path.path_to_file); + file_system_client.RenameFile(path.front(), dest_path.path_to_file); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } return Status::OK(); } - auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + auto directory_client = file_system_client.GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { if ((it + 1) == path.end()) { break; } - directoryClient = directoryClient.GetSubdirectoryClient(*it); + directory_client = directory_client.GetSubdirectoryClient(*it); ++it; } try { - directoryClient.RenameFile(it->data(), dest_path.path_to_file); + directory_client.RenameFile(it->data(), dest_path.path_to_file); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - } else if (DirExists(dfs_endpoint_url + src_path.full_path).ValueOrDie()) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(src_path.container); + } else if (dir_exists) { + auto file_system_client = gen2_client_->GetFileSystemClient(src_path.container); auto path = src_path.path_to_file_parts; if (path.size() == 1) { try { - fileSystemClient.RenameDirectory(path.front(), dest_path.path_to_file); + file_system_client.RenameDirectory(path.front(), dest_path.path_to_file); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } return Status::OK(); } - auto directoryClient = fileSystemClient.GetDirectoryClient(path.front()); + auto directory_client = file_system_client.GetDirectoryClient(path.front()); std::vector::const_iterator it = path.begin(); std::advance(it, 1); while (it != path.end()) { if ((it + 1) == path.end()) { break; } - directoryClient = directoryClient.GetSubdirectoryClient(*it); + directory_client = directory_client.GetSubdirectoryClient(*it); ++it; } try { - directoryClient.RenameSubdirectory(it->data(), dest_path.path_to_file); + directory_client.RenameSubdirectory(it->data(), dest_path.path_to_file); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } else { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(src_path.full_path); } return Status::OK(); } @@ -1081,36 +1202,42 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto src_path, AzurePath::FromString(src)); ARROW_ASSIGN_OR_RAISE(auto dest_path, AzurePath::FromString(dest)); - if (src_path.empty() || dest_path.empty() || src_path.path_to_file.empty() || - dest_path.path_to_file.empty()) { - return Status::IOError("Cannot copy file, file doesn't exist at src"); + if (src_path.empty() || src_path.path_to_file.empty()) { + return ::arrow::fs::internal::NotAFile(src_path.full_path); + } + if (dest_path.empty() || dest_path.path_to_file.empty()) { + return ::arrow::fs::internal::PathNotFound(dest_path.full_path); } - if (!(FileExists(dfs_endpoint_url + src_path.full_path)).ValueOrDie()) { - return Status::IOError("Cannot copy file, file doesn't exist at src"); + ARROW_ASSIGN_OR_RAISE(auto response, + FileExists(dfs_endpoint_url_ + src_path.full_path)); + if (!response) { + return ::arrow::fs::internal::NotAFile(src_path.full_path); } - if (DirExists(dfs_endpoint_url + dest_path.full_path).ValueOrDie()) { - return Status::IOError("Cannot copy file, Invalid destination path"); + ARROW_ASSIGN_OR_RAISE(response, DirExists(dfs_endpoint_url_ + dest_path.full_path)); + if (response) { + return Status::IOError( + "Cannot copy file, Invalid Azure Blob Storage destination path"); } - if (!isHierarchicalNamespaceEnabled) { + if (!is_hierarchical_namespace_enabled_) { if (src_path.path_to_file_parts.size() > 1 || dest_path.path_to_file_parts.size() > 1) { return Status::IOError( - "Invalid path provided, " - "hierarchical namespace not enabled"); + "Invalid Azure Blob Storage path provided, " + "hierarchical namespace not enabled in storage account"); } if (dest_path.empty() || dest_path.path_to_file_parts.empty()) { - return Status::IOError("Invalid path provided at destination"); + return ::arrow::fs::internal::PathNotFound(dest_path.full_path); } if (src_path == dest_path) { return Status::OK(); } - auto containerClient = gen1Client_->GetBlobContainerClient(dest_path.container); - auto fileClient = containerClient.GetBlobClient(dest_path.path_to_file); + auto container_client = gen1_client_->GetBlobContainerClient(dest_path.container); + auto file_client = container_client.GetBlobClient(dest_path.path_to_file); try { - auto response = fileClient.StartCopyFromUri(blob_endpoint_url + src); + file_client.StartCopyFromUri(blob_endpoint_url_ + src); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } @@ -1120,13 +1247,15 @@ class AzureBlobFileSystem::Impl if (dest_path.has_parent()) { AzurePath parent_path = dest_path.parent(); if (parent_path.path_to_file.empty()) { - if (!ContainerExists(parent_path.container).ValueOrDie()) { + ARROW_ASSIGN_OR_RAISE(response, ContainerExists(parent_path.container)); + if (!response) { return Status::IOError("Cannot copy file '", src_path.full_path, "': parent directory of destination does not exist"); } } else { - auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); - if (!(exists.ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(response, + DirExists(dfs_endpoint_url_ + parent_path.full_path)); + if (!response) { return Status::IOError("Cannot copy file '", src_path.full_path, "': parent directory of destination does not exist"); } @@ -1135,13 +1264,13 @@ class AzureBlobFileSystem::Impl if (src_path == dest_path) { return Status::OK(); } - auto containerClient = gen1Client_->GetBlobContainerClient(dest_path.container); - auto fileClient = containerClient.GetBlobClient(dest_path.path_to_file); + auto container_client = gen1_client_->GetBlobContainerClient(dest_path.container); + auto file_client = container_client.GetBlobClient(dest_path.path_to_file); try { if (options_.credentials_kind == AzureCredentialsKind::Sas) { - fileClient.StartCopyFromUri(blob_endpoint_url + src + options_.sas_token); + file_client.StartCopyFromUri(blob_endpoint_url_ + src + options_.sas_token); } else { - fileClient.StartCopyFromUri(blob_endpoint_url + src); + file_client.StartCopyFromUri(blob_endpoint_url_ + src); } } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); @@ -1150,73 +1279,77 @@ class AzureBlobFileSystem::Impl } Status ListPaths(const std::string& container, const std::string& path, - std::vector* childrenDirs, - std::vector* childrenFiles, + std::vector* children_dirs, + std::vector* children_files, const bool allow_not_found = false) { - if (!isHierarchicalNamespaceEnabled) { + if (!is_hierarchical_namespace_enabled_) { try { - auto paths = gen1Client_->GetBlobContainerClient(container).ListBlobs(); + auto paths = gen1_client_->GetBlobContainerClient(container).ListBlobs(); for (auto p : paths.Blobs) { std::shared_ptr - pathClient_; - RETURN_NOT_OK( + path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name)); - childrenFiles->push_back(container + "/" + p.Name); + options_, dfs_endpoint_url_ + container + "/" + p.Name, container, + p.Name)); + children_files->push_back(container + "/" + p.Name); } - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { if (!allow_not_found) { - return Status::IOError("Path does not exists"); + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } return Status::OK(); } if (path.empty()) { try { - auto paths = gen2Client_->GetFileSystemClient(container).ListPaths(false); + auto paths = gen2_client_->GetFileSystemClient(container).ListPaths(false); for (auto p : paths.Paths) { std::shared_ptr - pathClient_; - RETURN_NOT_OK( + path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, - container, p.Name)); - if (pathClient_->GetProperties().Value.IsDirectory) { - childrenDirs->push_back(container + "/" + p.Name); + options_, dfs_endpoint_url_ + container + "/" + p.Name, container, + p.Name)); + if (path_client->GetProperties().Value.IsDirectory) { + children_dirs->push_back(container + "/" + p.Name); } else { - childrenFiles->push_back(container + "/" + p.Name); + children_files->push_back(container + "/" + p.Name); } } - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { if (!allow_not_found) { - return Status::IOError("Path does not exists"); + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } return Status::OK(); } std::vector dirs = internal::SplitAbstractPath(path); try { - Azure::Storage::Files::DataLake::DataLakeDirectoryClient dirClient = - gen2Client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); + Azure::Storage::Files::DataLake::DataLakeDirectoryClient dir_client = + gen2_client_->GetFileSystemClient(container).GetDirectoryClient(dirs.front()); for (auto dir = dirs.begin() + 1; dir < dirs.end(); ++dir) { - dirClient = dirClient.GetSubdirectoryClient(*dir); + dir_client = dir_client.GetSubdirectoryClient(*dir); } - auto paths = dirClient.ListPaths(false); + auto paths = dir_client.ListPaths(false); for (auto p : paths.Paths) { - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, dfs_endpoint_url + container + "/" + p.Name, container, - p.Name)); - if (pathClient_->GetProperties().Value.IsDirectory) { - childrenDirs->push_back(container + "/" + p.Name); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, + InitPathClient( + options_, dfs_endpoint_url_ + container + "/" + p.Name, container, + p.Name)); + if (path_client->GetProperties().Value.IsDirectory) { + children_dirs->push_back(container + "/" + p.Name); } else { - childrenFiles->push_back(container + "/" + p.Name); + children_files->push_back(container + "/" + p.Name); } } - } catch (std::exception const& e) { + } catch (const Azure::Storage::StorageException& exception) { if (!allow_not_found) { - return Status::IOError("Path does not exists"); + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } return Status::OK(); @@ -1224,34 +1357,29 @@ class AzureBlobFileSystem::Impl Status Walk(const FileSelector& select, const std::string& container, const std::string& path, int nesting_depth, std::vector* out) { - std::vector childrenDirs; - std::vector childrenFiles; + std::vector children_dirs; + std::vector children_files; - Status st = - ListPaths(container, path, &childrenDirs, &childrenFiles, select.allow_not_found); - if (!st.ok()) { - return st; - } + RETURN_NOT_OK(ListPaths(container, path, &children_dirs, &children_files, + select.allow_not_found)); - for (const auto& childFile : childrenFiles) { + for (const auto& child_file : children_files) { FileInfo info; - // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - RETURN_NOT_OK(GetProperties(dfs_endpoint_url + childFile, &properties)); - PathInfoToFileInfo(childFile, FileType::File, properties.FileSize, + RETURN_NOT_OK(GetProperties(dfs_endpoint_url_ + child_file, &properties)); + PathInfoToFileInfo(child_file, FileType::File, properties.FileSize, properties.LastModified, &info); out->push_back(std::move(info)); } - for (const auto& childDir : childrenDirs) { + for (const auto& child_dir : children_dirs) { FileInfo info; - // std::string url = gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; - RETURN_NOT_OK(GetProperties(dfs_endpoint_url + childDir, &properties)); - PathInfoToFileInfo(childDir, FileType::Directory, -1, properties.LastModified, + RETURN_NOT_OK(GetProperties(dfs_endpoint_url_ + child_dir, &properties)); + PathInfoToFileInfo(child_dir, FileType::Directory, -1, properties.LastModified, &info); out->push_back(std::move(info)); if (select.recursive && nesting_depth < select.max_recursion) { - const auto src = internal::RemoveTrailingSlash(childDir); + const auto src = internal::RemoveTrailingSlash(child_dir); auto first_sep = src.find_first_of("/"); std::string s = std::string(src.substr(first_sep + 1)); RETURN_NOT_OK(Walk(select, container, s, nesting_depth + 1, out)); @@ -1264,53 +1392,64 @@ class AzureBlobFileSystem::Impl const std::string& s, Azure::Storage::Files::DataLake::Models::PathProperties* properties) { ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, s, path.container, path.path_to_file)); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( + options_, s, path.container, path.path_to_file)); if (path.path_to_file.empty()) { - auto fileSystemClient = gen2Client_->GetFileSystemClient(path.container); - auto props = fileSystemClient.GetProperties().Value; + auto file_system_client = gen2_client_->GetFileSystemClient(path.container); + try { + auto props = file_system_client.GetProperties().Value; + properties->LastModified = props.LastModified; + properties->Metadata = props.Metadata; + properties->ETag = props.ETag; + properties->FileSize = -1; + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + return Status::OK(); + } + try { + auto props = path_client->GetProperties().Value; + properties->FileSize = props.FileSize; properties->LastModified = props.LastModified; properties->Metadata = props.Metadata; properties->ETag = props.ETag; - properties->FileSize = -1; - return Status::OK(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - auto props = pathClient_->GetProperties().Value; - properties->FileSize = props.FileSize; - properties->LastModified = props.LastModified; - properties->Metadata = props.Metadata; - properties->ETag = props.ETag; return Status::OK(); } Status DeleteDirContents(const std::string& container, const std::string& path, const std::vector& path_to_file_parts) { - std::vector childrenDirs; - std::vector childrenFiles; + std::vector children_dirs; + std::vector children_files; - Status st = ListPaths(container, path, &childrenDirs, &childrenFiles); - if (!st.ok()) { - return st; - } - for (const auto& childFile : childrenFiles) { - ARROW_ASSIGN_OR_RAISE(auto filePath, AzurePath::FromString(childFile)); - RETURN_NOT_OK(DeleteFile(filePath.container, filePath.path_to_file_parts)); + RETURN_NOT_OK(ListPaths(container, path, &children_dirs, &children_files)); + + for (const auto& child_file : children_files) { + ARROW_ASSIGN_OR_RAISE(auto file_path, AzurePath::FromString(child_file)); + RETURN_NOT_OK(DeleteFile(file_path.container, file_path.path_to_file_parts)); } - for (const auto& childDir : childrenDirs) { - ARROW_ASSIGN_OR_RAISE(auto dirPath, AzurePath::FromString(childDir)); - RETURN_NOT_OK(DeleteDir(dirPath.container, dirPath.path_to_file_parts)); + for (const auto& child_dir : children_dirs) { + ARROW_ASSIGN_OR_RAISE(auto dir_path, AzurePath::FromString(child_dir)); + RETURN_NOT_OK(DeleteDir(dir_path.container, dir_path.path_to_file_parts)); } return Status::OK(); } Result> ListContainers() { - auto outcome = gen2Client_->ListFileSystems(); - std::vector containers; - for (auto container : outcome.FileSystems) { - containers.push_back(container.Name); + try { + auto outcome = gen2_client_->ListFileSystems(); + std::vector containers; + for (auto container : outcome.FileSystems) { + containers.push_back(container.Name); + } + return containers; + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); } - return containers; } Result> OpenInputFile(const std::string& s, @@ -1318,29 +1457,32 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty()) { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } - if (!isHierarchicalNamespaceEnabled) { + if (!is_hierarchical_namespace_enabled_) { if (path.path_to_file_parts.size() > 1) { return Status::IOError( - "Invalid path provided," - " hierarchical namespace not enabled"); + "Invalid Azure Blob Storage path provided," + " hierarchical namespace not enabled in storage account"); } } - if (!(FileExists(dfs_endpoint_url + path.full_path).ValueOrDie())) { - return Status::IOError("Invalid path provided"); + ARROW_ASSIGN_OR_RAISE(auto response, FileExists(dfs_endpoint_url_ + path.full_path)); + if (!response) { + return ::arrow::fs::internal::PathNotFound(path.full_path); } - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, dfs_endpoint_url + path.full_path, path.container, - path.path_to_file)); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( + options_, dfs_endpoint_url_ + path.full_path, path.container, + path.path_to_file)); - std::shared_ptr fileClient_; - RETURN_NOT_OK(InitPathClient( - fileClient_, options_, dfs_endpoint_url + path.full_path, path.container, - path.path_to_file)); + std::shared_ptr file_client; + ARROW_ASSIGN_OR_RAISE( + file_client, InitPathClient( + options_, dfs_endpoint_url_ + path.full_path, path.container, + path.path_to_file)); - auto ptr = std::make_shared(pathClient_, fileClient_, + auto ptr = std::make_shared(path_client, file_client, fs->io_context(), path); RETURN_NOT_OK(ptr->Init()); return ptr; @@ -1352,51 +1494,52 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty() || path.path_to_file.empty()) { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } - std::string endpoint_url = dfs_endpoint_url; - if (!isHierarchicalNamespaceEnabled) { + std::string endpoint_url = dfs_endpoint_url_; + if (!is_hierarchical_namespace_enabled_) { if (path.path_to_file_parts.size() > 1) { return Status::IOError( "Invalid path provided," " hierarchical namespace not enabled"); } - endpoint_url = blob_endpoint_url; + endpoint_url = blob_endpoint_url_; } - if (DirExists(dfs_endpoint_url + path.full_path).ValueOrDie()) { - return Status::IOError("Invalid path provided"); + ARROW_ASSIGN_OR_RAISE(auto response, DirExists(dfs_endpoint_url_ + path.full_path)); + if (response) { + return ::arrow::fs::internal::PathNotFound(path.full_path); } - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); + std::shared_ptr file_client; + ARROW_ASSIGN_OR_RAISE( + file_client, + InitPathClient( + options_, endpoint_url + path.full_path, path.container, path.path_to_file)); - std::shared_ptr fileClient_; - RETURN_NOT_OK(InitPathClient( - fileClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); - - std::shared_ptr blobClient_; - RETURN_NOT_OK(InitPathClient( - blobClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); + std::shared_ptr blob_client; + ARROW_ASSIGN_OR_RAISE( + blob_client, + InitPathClient( + options_, endpoint_url + path.full_path, path.container, path.path_to_file)); if (path.has_parent()) { AzurePath parent_path = path.parent(); if (parent_path.path_to_file.empty()) { - if (!ContainerExists(parent_path.container).ValueOrDie()) { + ARROW_ASSIGN_OR_RAISE(response, ContainerExists(parent_path.container)); + if (!response) { return Status::IOError("Cannot write to file '", path.full_path, "': parent directory does not exist"); } } else { - auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); - if (!(exists.ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(response, + DirExists(dfs_endpoint_url_ + parent_path.full_path)); + if (!response) { return Status::IOError("Cannot write to file '", path.full_path, "': parent directory does not exist"); } } } - auto ptr = std::make_shared(pathClient_, fileClient_, blobClient_, isHierarchicalNamespaceEnabled, + auto ptr = std::make_shared(file_client, blob_client, + is_hierarchical_namespace_enabled_, fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; @@ -1408,51 +1551,58 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(s)); if (path.empty() || path.path_to_file.empty()) { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } - std::string endpoint_url = dfs_endpoint_url; - if (!isHierarchicalNamespaceEnabled) { + std::string endpoint_url = dfs_endpoint_url_; + if (!is_hierarchical_namespace_enabled_) { if (path.path_to_file_parts.size() > 1) { return Status::IOError( - "Invalid path provided," - " hierarchical namespace not enabled"); + "Invalid Azure Blob Storage path provided," + " hierarchical namespace not enabled in storage account"); } - endpoint_url = blob_endpoint_url; - } - if (DirExists(dfs_endpoint_url + path.full_path).ValueOrDie()) { - return Status::IOError("Invalid path provided"); - } - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); - - std::shared_ptr fileClient_; - RETURN_NOT_OK(InitPathClient( - fileClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); - - std::shared_ptr blobClient_; - RETURN_NOT_OK(InitPathClient( - blobClient_, options_, endpoint_url + path.full_path, path.container, - path.path_to_file)); + endpoint_url = blob_endpoint_url_; + } + ARROW_ASSIGN_OR_RAISE(auto response, DirExists(dfs_endpoint_url_ + path.full_path)); + if (response) { + return ::arrow::fs::internal::PathNotFound(path.full_path); + } + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, + InitPathClient( + options_, endpoint_url + path.full_path, path.container, path.path_to_file)); + + std::shared_ptr file_client; + ARROW_ASSIGN_OR_RAISE( + file_client, + InitPathClient( + options_, endpoint_url + path.full_path, path.container, path.path_to_file)); + + std::shared_ptr blob_client; + ARROW_ASSIGN_OR_RAISE( + blob_client, + InitPathClient( + options_, endpoint_url + path.full_path, path.container, path.path_to_file)); if (path.has_parent()) { AzurePath parent_path = path.parent(); if (parent_path.path_to_file.empty()) { - if (!ContainerExists(parent_path.container).ValueOrDie()) { + ARROW_ASSIGN_OR_RAISE(response, ContainerExists(parent_path.container)); + if (!response) { return Status::IOError("Cannot write to file '", path.full_path, "': parent directory does not exist"); } } else { - auto exists = DirExists(dfs_endpoint_url + parent_path.full_path); - if (!(exists.ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(response, + DirExists(dfs_endpoint_url_ + parent_path.full_path)); + if (!response) { return Status::IOError("Cannot write to file '", path.full_path, "': parent directory does not exist"); } } } - auto ptr = std::make_shared(pathClient_, fileClient_, blobClient_, isHierarchicalNamespaceEnabled, + auto ptr = std::make_shared(path_client, file_client, blob_client, + is_hierarchical_namespace_enabled_, fs->io_context(), path, metadata); RETURN_NOT_OK(ptr->Init()); return ptr; @@ -1469,27 +1619,30 @@ class AzureBlobFileSystem::Impl ARROW_ASSIGN_OR_RAISE(auto path, AzurePath::FromString(info.path())); - if (!isHierarchicalNamespaceEnabled) { + if (!is_hierarchical_namespace_enabled_) { if (path.path_to_file_parts.size() > 1) { return Status::IOError( - "Invalid path provided, hierarchical namespace" - " not enabled"); + "Invalid Azure Blob Storage path provided, hierarchical namespace" + " not enabled in storage account"); } } - if (!(FileExists(dfs_endpoint_url + info.path())).ValueOrDie()) { - return Status::IOError("Invalid path provided"); + ARROW_ASSIGN_OR_RAISE(auto response, FileExists(dfs_endpoint_url_ + info.path())); + if (!response) { + return ::arrow::fs::internal::PathNotFound(info.path()); } - std::shared_ptr pathClient_; - RETURN_NOT_OK(InitPathClient( - pathClient_, options_, dfs_endpoint_url + info.path(), path.container, - path.path_to_file)); + std::shared_ptr path_client; + ARROW_ASSIGN_OR_RAISE( + path_client, InitPathClient( + options_, dfs_endpoint_url_ + info.path(), path.container, + path.path_to_file)); - std::shared_ptr fileClient_; - RETURN_NOT_OK(InitPathClient( - fileClient_, options_, dfs_endpoint_url + info.path(), path.container, - path.path_to_file)); + std::shared_ptr file_client; + ARROW_ASSIGN_OR_RAISE( + file_client, InitPathClient( + options_, dfs_endpoint_url_ + info.path(), path.container, + path.path_to_file)); - auto ptr = std::make_shared(pathClient_, fileClient_, + auto ptr = std::make_shared(path_client, file_client, fs->io_context(), path, info.size()); RETURN_NOT_OK(ptr->Init()); return ptr; @@ -1521,9 +1674,9 @@ bool AzureBlobFileSystem::Equals(const FileSystem& other) const { if (other.type_name() != type_name()) { return false; } - const auto& azurefs = + const auto& azure_fs = ::arrow::internal::checked_cast(other); - return options().Equals(azurefs.options()); + return options().Equals(azure_fs.options()); } AzureOptions AzureBlobFileSystem::options() const { return impl_->options(); } @@ -1533,7 +1686,7 @@ Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { FileInfo info; info.set_path(s); - if (!impl_->isHierarchicalNamespaceEnabled) { + if (!impl_->is_hierarchical_namespace_enabled_) { if (path.path_to_file_parts.size() > 1) { info.set_type(FileType::NotFound); return info; @@ -1556,18 +1709,19 @@ Result AzureBlobFileSystem::GetFileInfo(const std::string& s) { } else { // It's an object ARROW_ASSIGN_OR_RAISE(bool file_exists, - impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)); + impl_->FileExists(impl_->dfs_endpoint_url_ + path.full_path)); if (file_exists) { // "File" object found Azure::Storage::Files::DataLake::Models::PathProperties properties; RETURN_NOT_OK( - impl_->GetProperties(impl_->dfs_endpoint_url + path.full_path, &properties)); + impl_->GetProperties(impl_->dfs_endpoint_url_ + path.full_path, &properties)); FileObjectToInfo(properties, &info); return info; } // Not found => perhaps it's a "directory" - auto is_dir = impl_->DirExists(impl_->dfs_endpoint_url + path.full_path); - if (is_dir.ValueOrDie()) { + ARROW_ASSIGN_OR_RAISE(auto is_dir, + impl_->DirExists(impl_->dfs_endpoint_url_ + path.full_path)); + if (is_dir) { info.set_type(FileType::Directory); } else { info.set_type(FileType::NotFound); @@ -1586,10 +1740,9 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele ARROW_ASSIGN_OR_RAISE(auto containers, impl_->ListContainers()); for (const auto& container : containers) { FileInfo info; - // std::string url = impl_->gen2Client_->GetUrl(); Azure::Storage::Files::DataLake::Models::PathProperties properties; RETURN_NOT_OK( - impl_->GetProperties(impl_->dfs_endpoint_url + container, &properties)); + impl_->GetProperties(impl_->dfs_endpoint_url_ + container, &properties)); PathInfoToFileInfo(container, FileType::Directory, -1, properties.LastModified, &info); results.push_back(std::move(info)); @@ -1600,33 +1753,36 @@ Result AzureBlobFileSystem::GetFileInfo(const FileSelector& sele return results; } - if (!impl_->isHierarchicalNamespaceEnabled) { + if (!impl_->is_hierarchical_namespace_enabled_) { if (base_path.path_to_file_parts.size() > 1) { if (!select.allow_not_found) { return Status::IOError( - "Invalid path provided, hierarchical namespace not" - " enabled"); + "Invalid Azure Blob Storage path provided, hierarchical namespace not" + " enabled in storage account"); } return results; } } - if (base_path.path_to_file.empty() && - !(impl_->ContainerExists(base_path.container).ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(auto response, impl_->ContainerExists(base_path.container)); + if (base_path.path_to_file.empty() && !response) { if (!select.allow_not_found) { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(base_path.container); } return results; } - if (impl_->FileExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie()) { - return Status::IOError("Invalid path provided"); + ARROW_ASSIGN_OR_RAISE( + response, impl_->FileExists(impl_->dfs_endpoint_url_ + base_path.full_path)); + if (response) { + return ::arrow::fs::internal::PathNotFound(base_path.full_path); } - if (!(base_path.path_to_file.empty()) && - !(impl_->DirExists(impl_->dfs_endpoint_url + base_path.full_path).ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(response, + impl_->DirExists(impl_->dfs_endpoint_url_ + base_path.full_path)); + if (!(base_path.path_to_file.empty()) && !response) { if (!select.allow_not_found) { - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(base_path.full_path); } return results; } @@ -1643,15 +1799,17 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { if (path.empty()) { return Status::IOError("Cannot create directory, root path given"); } - if ((impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)).ValueOrDie()) { - return Status::IOError("Cannot create directory, file exists at path"); + ARROW_ASSIGN_OR_RAISE(auto response, + impl_->FileExists(impl_->dfs_endpoint_url_ + path.full_path)); + if (response) { + return Status::IOError("Cannot create directory, file exists at the specified path"); } if (path.path_to_file.empty()) { // Create container return impl_->CreateContainer(path.container); } // Hierarchical namespace not enabled type storage accounts - if (!impl_->isHierarchicalNamespaceEnabled) { + if (!impl_->is_hierarchical_namespace_enabled_) { if (!path.path_to_file.empty()) { return Status::IOError( "Cannot create directory, " @@ -1676,14 +1834,15 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { if (path.has_parent()) { AzurePath parent_path = path.parent(); if (parent_path.path_to_file.empty()) { - auto exists = impl_->ContainerExists(parent_path.container); - if (!(exists.ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(auto exists, impl_->ContainerExists(parent_path.container)); + if (!exists) { return Status::IOError("Cannot create directory '", path.full_path, "': parent directory does not exist"); } } else { - auto exists = impl_->DirExists(impl_->dfs_endpoint_url + parent_path.full_path); - if (!(exists.ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(auto exists, impl_->DirExists(impl_->dfs_endpoint_url_ + + parent_path.full_path)); + if (!exists) { return Status::IOError("Cannot create directory '", path.full_path, "': parent directory does not exist"); } @@ -1701,12 +1860,14 @@ Status AzureBlobFileSystem::DeleteDir(const std::string& s) { if (path.path_to_file.empty()) { return impl_->DeleteContainer(path.container); } - if ((impl_->FileExists(impl_->dfs_endpoint_url + path.full_path)).ValueOrDie()) { - return Status::IOError("Cannot delete directory, file exists at path"); + ARROW_ASSIGN_OR_RAISE(auto response, + impl_->FileExists(impl_->dfs_endpoint_url_ + path.full_path)); + if (response) { + return Status::IOError("Cannot delete directory, file exists at the specified path"); } // Hierarchical namespace not enabled type storage accounts - if (!impl_->isHierarchicalNamespaceEnabled) { + if (!impl_->is_hierarchical_namespace_enabled_) { if (!path.path_to_file.empty()) { return Status::IOError( "Cannot delete directory, storage" @@ -1723,30 +1884,33 @@ Status AzureBlobFileSystem::DeleteDirContents(const std::string& s, bool missing if (missing_dir_ok) { return Status::OK(); } - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } - if (path.path_to_file.empty() && - !(impl_->ContainerExists(path.container).ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(auto response, impl_->ContainerExists(path.container)); + if (path.path_to_file.empty() && !response) { if (missing_dir_ok) { return Status::OK(); } - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.container); } - if (impl_->FileExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie()) { + ARROW_ASSIGN_OR_RAISE(response, + impl_->FileExists(impl_->dfs_endpoint_url_ + path.full_path)); + if (response) { if (missing_dir_ok) { return Status::OK(); } - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } - if (!(path.path_to_file.empty()) && - !(impl_->DirExists(impl_->dfs_endpoint_url + path.full_path).ValueOrDie())) { + ARROW_ASSIGN_OR_RAISE(response, + impl_->DirExists(impl_->dfs_endpoint_url_ + path.full_path)); + if (!(path.path_to_file.empty()) && !response) { if (missing_dir_ok) { return Status::OK(); } - return Status::IOError("Invalid path provided"); + return ::arrow::fs::internal::PathNotFound(path.full_path); } return impl_->DeleteDirContents(path.container, path.path_to_file, diff --git a/cpp/src/arrow/filesystem/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h index b77d9d1e05ab1..ecdd60dd8d16e 100644 --- a/cpp/src/arrow/filesystem/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -73,37 +73,38 @@ struct ARROW_EXPORT AzureOptions { AzureOptions(); - std::string GetAccountNameFromConnectionString(const std::string& connectionString); + Result GetAccountNameFromConnectionString( + const std::string& connectionString); - void ConfigureAnonymousCredentials(const std::string& account_name); + Status ConfigureAnonymousCredentials(const std::string& account_name); - void ConfigureAccountKeyCredentials(const std::string& account_name, - const std::string& account_key); + Status ConfigureAccountKeyCredentials(const std::string& account_name, + const std::string& account_key); - void ConfigureConnectionStringCredentials(const std::string& connection_string); + Status ConfigureConnectionStringCredentials(const std::string& connection_string); - void ConfigureServicePrincipleCredentials(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret); + Status ConfigureServicePrincipleCredentials(const std::string& account_name, + const std::string& tenant_id, + const std::string& client_id, + const std::string& client_secret); - void ConfigureSasCredentials(const std::string& sas_token); + Status ConfigureSasCredentials(const std::string& sas_token); bool Equals(const AzureOptions& other) const; - static AzureOptions FromAnonymous(const std::string account_name); + static Result FromAnonymous(const std::string& account_name); - static AzureOptions FromAccountKey(const std::string& account_name, - const std::string& account_key); + static Result FromAccountKey(const std::string& account_name, + const std::string& account_key); - static AzureOptions FromConnectionString(const std::string& connection_string); + // https://docs.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string + static Result FromConnectionString(const std::string& connection_string); - static AzureOptions FromServicePrincipleCredential(const std::string& account_name, - const std::string& tenant_id, - const std::string& client_id, - const std::string& client_secret); + static Result FromServicePrincipleCredential( + const std::string& account_name, const std::string& tenant_id, + const std::string& client_id, const std::string& client_secret); - static AzureOptions FromSas(const std::string& uri); + static Result FromSas(const std::string& uri); static Result FromUri(const ::arrow::internal::Uri& uri, std::string* out_path = NULLPTR); diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 88bdcea8160a5..62abacb66fad6 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -128,8 +128,8 @@ class TestAzureFileSystem : public ::testing::Test { options_.account_blob_url + "container/somefile", options_.storage_credentials_provider); std::string s = "some data"; - file_client->UploadFrom(const_cast(reinterpret_cast(s.data())), - s.size()); + file_client->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); } void TearDown() override { @@ -168,7 +168,8 @@ class TestAzureFileSystem : public ::testing::Test { size, download_options) .Value; auto buf_data = std::move(buf->get()); - auto expected_data = std::make_shared(reinterpret_cast(expected.data()), expected.size()); + auto expected_data = std::make_shared( + reinterpret_cast(expected.data()), expected.size()); AssertBufferEqual(*buf_data, *expected_data); } }; @@ -183,35 +184,52 @@ TEST_F(TestAzureFileSystem, FromUri) { ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); // Sas Token - ASSERT_OK(uri.Parse("https://testcontainer.blob.core.windows.net/?dummy_sas_token")); + ASSERT_OK(uri.Parse( + "https://testcontainer.blob.core.windows.net/?dummy_sas_key=dummy_value")); ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); - ASSERT_EQ(options.sas_token, "?dummy_sas_token"); + ASSERT_EQ(options.account_blob_url, "https://testcontainer.blob.core.windows.net/"); + ASSERT_EQ(options.sas_token, "?dummy_sas_key=dummy_value"); } TEST_F(TestAzureFileSystem, FromAccountKey) { - AzureOptions options = AzureOptions::FromAccountKey(GetAzuriteEnv()->account_name(), GetAzuriteEnv()->account_key()); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::StorageCredentials); + auto options = AzureOptions::FromAccountKey(GetAzuriteEnv()->account_name(), + GetAzuriteEnv()->account_key()) + .ValueOrDie(); + ASSERT_EQ(options.credentials_kind, + arrow::fs::AzureCredentialsKind::StorageCredentials); ASSERT_NE(options.storage_credentials_provider, nullptr); } TEST_F(TestAzureFileSystem, FromConnectionString) { - AzureOptions options = AzureOptions::FromConnectionString("DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;"); + auto options = + AzureOptions::FromConnectionString( + "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=" + "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/" + "KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;") + .ValueOrDie(); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ConnectionString); ASSERT_NE(options.connection_string, ""); } TEST_F(TestAzureFileSystem, FromServicePrincipleCredential) { - AzureOptions options = AzureOptions::FromServicePrincipleCredential("dummy_account_name", "dummy_tenant_id", "dummy_client_id", "dummy_client_secret"); - ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::ServicePrincipleCredentials); + auto options = AzureOptions::FromServicePrincipleCredential( + "dummy_account_name", "dummy_tenant_id", "dummy_client_id", + "dummy_client_secret") + .ValueOrDie(); + ASSERT_EQ(options.credentials_kind, + arrow::fs::AzureCredentialsKind::ServicePrincipleCredentials); ASSERT_NE(options.service_principle_credentials_provider, nullptr); } TEST_F(TestAzureFileSystem, FromSas) { - AzureOptions options = AzureOptions::FromSas("https://testcontainer.blob.core.windows.net/?dummy_sas_token"); + auto options = + AzureOptions::FromSas( + "https://testcontainer.blob.core.windows.net/?dummy_sas_key=dummy_value") + .ValueOrDie(); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); - ASSERT_NE(options.sas_token, ""); + ASSERT_EQ(options.sas_token, "?dummy_sas_key=dummy_value"); } TEST_F(TestAzureFileSystem, CreateDir) { @@ -321,12 +339,14 @@ TEST_F(TestAzureFileSystem, Move) { ASSERT_RAISES(IOError, fs_->Move("container/somedir/subdir", "container/newdir/newsub")); ASSERT_RAISES(IOError, fs_->Move("container/emptydir", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/nonexistent-directory", "container/base.txt")); + ASSERT_RAISES(IOError, + fs_->Move("container/nonexistent-directory", "container/base.txt")); ASSERT_OK_AND_ASSIGN(auto res, fs_->OpenOutputStream("container/somefile")); ASSERT_OK(res->Write("Changed the data")); ASSERT_RAISES(IOError, fs_->Move("container/base.txt", "container/somefile")); ASSERT_RAISES(IOError, fs_->Move("container/somefile", "container/base.txt")); - ASSERT_RAISES(IOError, fs_->Move("container/nonexistent-file.txt", "container/non-existentdir")); + ASSERT_RAISES(IOError, + fs_->Move("container/nonexistent-file.txt", "container/non-existentdir")); } TEST_F(TestAzureFileSystem, CopyFile) { @@ -486,7 +506,8 @@ TEST_F(TestAzureFileSystem, OpenAppendStream) { ASSERT_OK_AND_ASSIGN(stream, fs_->OpenAppendStream("container/newfile1")); ASSERT_OK(stream->Write(", more data")); ASSERT_OK(stream->Close()); - AssertObjectContents(gen2_client_.get(), "container", "newfile1", "append data, more data"); + AssertObjectContents(gen2_client_.get(), "container", "newfile1", + "append data, more data"); } TEST_F(TestAzureFileSystem, DeleteDirContents) { From b15a6b1c149160af85ec09f91544b7115e914603 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Wed, 27 Jul 2022 01:22:31 +0530 Subject: [PATCH 29/34] Fixed OpenOutputStream --- cpp/src/arrow/filesystem/azurefs.cc | 106 ++++++++++++++++++----- cpp/src/arrow/filesystem/azurefs_test.cc | 5 +- 2 files changed, 83 insertions(+), 28 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 0f983e1df7962..6161224865931 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -211,7 +211,7 @@ Result AzureOptions::FromUri(const Uri& uri, std::string* out_path return Status::IOError("Missing container in Azure Blob Storage URI: '", uri.ToString(), "'"); } - std::string full_path; + std::string full_path = path_to_blob; // account_name = accountName account_name = host.substr(0, pos); if (full_path.empty()) { @@ -302,10 +302,18 @@ struct AzurePath { } // Removes scheme, host and port from the uri - static Status ExtractBlobPath(util::string_view* s) { - Uri uri; - RETURN_NOT_OK(uri.Parse(s->to_string())); - *s = uri.path(); + static Status ExtractBlobPath(util::string_view* src) { + std::string text = ".core.windows.net"; + auto pos = src->find(text); + if (pos == std::string::npos) { + return Status::IOError("Invalid Azure blob storage URI provided: ", src); + } + pos = src->find("/", pos); + if (pos == std::string::npos) { + *src = ""; + } else { + *src = src->substr(pos + 1); + } return Status::OK(); } @@ -582,13 +590,22 @@ class ObjectOutputStream final : public io::OutputStream { DCHECK_GE(content_length_, 0); return Status::OK(); } - try { - std::string s = ""; - file_client_->UploadFrom( - const_cast(reinterpret_cast(s.data())), s.size()); - content_length_ = 0; - } catch (const Azure::Storage::StorageException& exception) { - return Status::IOError(exception.RawResponse->GetReasonPhrase()); + if (is_hierarchical_namespace_enabled_) { + try { + file_client_->DeleteIfExists(); + file_client_->CreateIfNotExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else { + try { + std::string s = ""; + file_client_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + content_length_ = 0; + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } return Status::OK(); } @@ -642,7 +659,11 @@ class ObjectOutputStream final : public io::OutputStream { auto buffer_stream = std::make_unique( Azure::Core::IO::MemoryBodyStream( const_cast(reinterpret_cast(data)), nbytes)); + if (buffer_stream->Length() == 0) { + return Status::OK(); + } auto result = file_client_->Append(*buffer_stream, pos_); + pos_ += nbytes; file_client_->Flush(pos_); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); @@ -651,11 +672,21 @@ class ObjectOutputStream final : public io::OutputStream { try { auto append_data = static_cast((void*)data); auto res = blob_client_->GetBlockList().Value; - std::string text = std::to_string(rand()); - const std::string block_id = Azure::Core::Convert::Base64Encode( - std::vector(text.begin(), text.end())); + auto size = res.CommittedBlocks.size(); + std::string block_id; + { + block_id = std::to_string(size + 1); + size_t n = 8; + int precision = n - std::min(n, block_id.size()); + block_id.insert(0, precision, '0'); + } + block_id = Azure::Core::Convert::Base64Encode( + std::vector(block_id.begin(), block_id.end())); auto block_content = Azure::Core::IO::MemoryBodyStream( append_data, strlen(reinterpret_cast(append_data))); + if (block_content.Length() == 0) { + return Status::OK(); + } blob_client_->StageBlock(block_id, block_content); std::vector block_ids; for (auto block : res.CommittedBlocks) { @@ -663,10 +694,12 @@ class ObjectOutputStream final : public io::OutputStream { } block_ids.push_back(block_id); blob_client_->CommitBlockList(block_ids); + pos_ += nbytes; } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } + content_length_ += nbytes; return Status::OK(); } @@ -740,14 +773,23 @@ class ObjectAppendStream final : public io::OutputStream { return ::arrow::fs::internal::NotAFile(path_.full_path); } content_length_ = properties.Value.FileSize; + pos_ = content_length_; } catch (const Azure::Storage::StorageException& exception) { // new file - std::string s = ""; - try { - file_client_->UploadFrom( - const_cast(reinterpret_cast(s.data())), s.size()); - } catch (const Azure::Storage::StorageException& exception) { - return Status::IOError(exception.RawResponse->GetReasonPhrase()); + if (is_hierarchical_namespace_enabled_) { + try { + file_client_->CreateIfNotExists(); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } + } else { + std::string s = ""; + try { + file_client_->UploadFrom( + const_cast(reinterpret_cast(s.data())), s.size()); + } catch (const Azure::Storage::StorageException& exception) { + return Status::IOError(exception.RawResponse->GetReasonPhrase()); + } } content_length_ = 0; } @@ -805,7 +847,11 @@ class ObjectAppendStream final : public io::OutputStream { auto buffer_stream = std::make_unique( Azure::Core::IO::MemoryBodyStream( const_cast(reinterpret_cast(data)), nbytes)); + if (buffer_stream->Length() == 0) { + return Status::OK(); + } auto result = file_client_->Append(*buffer_stream, pos_); + pos_ += nbytes; file_client_->Flush(pos_); } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); @@ -814,11 +860,21 @@ class ObjectAppendStream final : public io::OutputStream { try { auto append_data = static_cast((void*)data); auto res = blob_client_->GetBlockList().Value; - std::string text = std::to_string(rand()); - const std::string block_id = Azure::Core::Convert::Base64Encode( - std::vector(text.begin(), text.end())); + auto size = res.CommittedBlocks.size(); + std::string block_id; + { + block_id = std::to_string(size + 1); + size_t n = 8; + int precision = n - std::min(n, block_id.size()); + block_id.insert(0, precision, '0'); + } + block_id = Azure::Core::Convert::Base64Encode( + std::vector(block_id.begin(), block_id.end())); auto block_content = Azure::Core::IO::MemoryBodyStream( append_data, strlen(reinterpret_cast(append_data))); + if (block_content.Length() == 0) { + return Status::OK(); + } blob_client_->StageBlock(block_id, block_content); std::vector block_ids; for (auto block : res.CommittedBlocks) { @@ -826,10 +882,12 @@ class ObjectAppendStream final : public io::OutputStream { } block_ids.push_back(block_id); blob_client_->CommitBlockList(block_ids); + pos_ += nbytes; } catch (const Azure::Storage::StorageException& exception) { return Status::IOError(exception.RawResponse->GetReasonPhrase()); } } + content_length_ += nbytes; return Status::OK(); } diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 62abacb66fad6..9b555fc667e8f 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -300,11 +300,8 @@ TEST_F(TestAzureFileSystem, GetFileInfo) { AssertFileInfo(fs_.get(), "", FileType::Directory); - ASSERT_OK_AND_ASSIGN(auto res, fs_->OpenOutputStream("container/base.txt")); - ASSERT_OK(res->Write("Base data")); - // "Files" - AssertFileInfo(fs_.get(), "container/base.txt", FileType::File); + AssertFileInfo(fs_.get(), "container/somefile", FileType::File); AssertFileInfo(fs_.get(), "container/nonexistent-file.txt", FileType::NotFound); } From a40a316311ac55aa266af92027a2c64e574fe6dd Mon Sep 17 00:00:00 2001 From: shefali singh Date: Thu, 4 Aug 2022 03:15:25 +0530 Subject: [PATCH 30/34] Added uri.Parse() --- cpp/src/arrow/filesystem/azurefs.cc | 80 ++++++++++-------------- cpp/src/arrow/filesystem/azurefs.h | 2 +- cpp/src/arrow/filesystem/azurefs_test.cc | 8 +-- 3 files changed, 38 insertions(+), 52 deletions(-) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 6161224865931..c1712a4b1a64a 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -75,17 +75,17 @@ Result AzureOptions::GetAccountNameFromConnectionString( std::string text = "AccountName="; auto pos_text = connection_string.find(text); if (pos_text == std::string::npos) { - return Status::IOError( + return Status::Invalid( "Cannot find account name in Azure Blob Storage connection string: '", connection_string, "'"); } - auto pos_colon = connection_string.find(';'); - pos_colon = connection_string.find(';', pos_colon + 1); - if (pos_colon == std::string::npos) { - return Status::IOError("Invalid Azure Blob Storage connection string: '", + auto pos_semicolon = connection_string.find(';'); + pos_semicolon = connection_string.find(';', pos_semicolon + 1); + if (pos_semicolon == std::string::npos) { + return Status::Invalid("Invalid Azure Blob Storage connection string: '", connection_string, "' passed"); } - std::string account_name = connection_string.substr(pos_text + text.size(), pos_colon); + std::string account_name = connection_string.substr(pos_text + text.size(), pos_semicolon); return account_name; } @@ -140,7 +140,7 @@ Status AzureOptions::ConfigureSasCredentials(const std::string& uri) { RETURN_NOT_OK(url.Parse(uri)); sas_token = "?" + url.query_string(); account_blob_url = url.scheme() + "://" + url.host() + kSep; - account_dfs_url = std::regex_replace(account_blob_url, std::regex(".blob"), ".dfs"); + account_dfs_url = std::regex_replace(account_blob_url, std::regex("[.]blob"), ".dfs"); credentials_kind = AzureCredentialsKind::Sas; return Status::OK(); } @@ -256,18 +256,20 @@ struct AzurePath { // path_to_file = testdir/testfile.txt // path_to_file_parts = [testdir, testfile.txt] - // Expected input here => s = synapsemlfs/testdir/testfile.txt + // Expected input here => s = synapsemlfs/testdir/testfile.txt, http://127.0.0.1/accountName/pathToBlob auto src = internal::RemoveTrailingSlash(s); - if ((src.find("127.0.0.1") != std::string::npos)) { + if (src.starts_with("https://127.0.0.1") || src.starts_with("http://127.0.0.1")) { RETURN_NOT_OK(FromLocalHostString(&src)); } - if (internal::IsLikelyUri(src)) { - RETURN_NOT_OK(ExtractBlobPath(&src)); + auto input_path = std::string(src.data()); + if (internal::IsLikelyUri(input_path)) { + RETURN_NOT_OK(ExtractBlobPath(&input_path)); + src = util::string_view(input_path); } src = internal::RemoveLeadingSlash(src); auto first_sep = src.find_first_of(kSep); if (first_sep == 0) { - return Status::IOError("Path cannot start with a separator ('", s, "')"); + return Status::IOError("Path cannot start with a separator ('", input_path, "')"); } if (first_sep == std::string::npos) { return AzurePath{std::string(src), std::string(src), "", {}}; @@ -283,51 +285,40 @@ struct AzurePath { static Status FromLocalHostString(util::string_view* src) { // src = http://127.0.0.1:10000/accountName/pathToBlob - auto port = src->find("127.0.0.1"); - // src = 127.0.0.1:10000/accountName/pathToBlob - *src = src->substr(port); - auto first_sep = src->find_first_of(kSep); - if (first_sep == std::string::npos) { + Uri uri; + RETURN_NOT_OK(uri.Parse(src->data())); + *src = internal::RemoveLeadingSlash(uri.path()); + if (src->empty()) { return Status::IOError("Missing account name in Azure Blob Storage URI"); } - // src = accountName/pathToBlob - *src = src->substr(first_sep + 1); - auto sec_sep = src->find_first_of(kSep); - if (sec_sep == std::string::npos) { - return Status::IOError("Missing container name in Azure Blob Storage URI"); + auto first_sep = src->find_first_of(kSep); + if (first_sep != std::string::npos) { + *src = src->substr(first_sep + 1); + } else { + *src = ""; } - // src = pathToBlob - *src = src->substr(sec_sep + 1); return Status::OK(); } // Removes scheme, host and port from the uri - static Status ExtractBlobPath(util::string_view* src) { - std::string text = ".core.windows.net"; - auto pos = src->find(text); - if (pos == std::string::npos) { - return Status::IOError("Invalid Azure blob storage URI provided: ", src); - } - pos = src->find("/", pos); - if (pos == std::string::npos) { - *src = ""; - } else { - *src = src->substr(pos + 1); - } + static Status ExtractBlobPath(std::string* src) { + Uri uri; + RETURN_NOT_OK(uri.Parse(*src)); + *src = uri.path(); return Status::OK(); } static Status Validate(const AzurePath* path) { - auto result = internal::ValidateAbstractPathParts(path->path_to_file_parts); - if (!result.ok()) { - return Status::Invalid(result.message(), " in path ", path->full_path); + auto status = internal::ValidateAbstractPathParts(path->path_to_file_parts); + if (!status.ok()) { + return Status::Invalid(status.message(), " in path ", path->full_path); } else { - return result; + return status; } } AzurePath parent() const { - DCHECK(!path_to_file_parts.empty()); + DCHECK(has_parent()); auto parent = AzurePath{"", container, "", path_to_file_parts}; parent.path_to_file_parts.pop_back(); parent.path_to_file = internal::JoinAbstractPath(parent.path_to_file_parts); @@ -351,13 +342,8 @@ struct AzurePath { template std::shared_ptr GetObjectMetadata(const ObjectResult& result) { auto md = std::make_shared(); - auto push = [&](std::string k, const std::string v) { - if (!v.empty()) { - md->Append(std::move(k), v); - } - }; for (auto prop : result) { - push(prop.first, prop.second); + md->Append(prop.first, prop.second); } return md; } diff --git a/cpp/src/arrow/filesystem/azurefs.h b/cpp/src/arrow/filesystem/azurefs.h index ecdd60dd8d16e..d58b76ab3748f 100644 --- a/cpp/src/arrow/filesystem/azurefs.h +++ b/cpp/src/arrow/filesystem/azurefs.h @@ -74,7 +74,7 @@ struct ARROW_EXPORT AzureOptions { AzureOptions(); Result GetAccountNameFromConnectionString( - const std::string& connectionString); + const std::string& connection_string); Status ConfigureAnonymousCredentials(const std::string& account_name); diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc index 9b555fc667e8f..6a9429ca05836 100644 --- a/cpp/src/arrow/filesystem/azurefs_test.cc +++ b/cpp/src/arrow/filesystem/azurefs_test.cc @@ -116,7 +116,7 @@ class TestAzureFileSystem : public ::testing::Test { void SetUp() override { ASSERT_THAT(GetAzuriteEnv(), NotNull()); - ASSERT_THAT(GetAzuriteEnv()->status(), Status::OK()); + ASSERT_OK(GetAzuriteEnv()->status()); MakeFileSystem(); auto file_system_client = gen2_client_->GetFileSystemClient("container"); @@ -185,11 +185,11 @@ TEST_F(TestAzureFileSystem, FromUri) { // Sas Token ASSERT_OK(uri.Parse( - "https://testcontainer.blob.core.windows.net/?dummy_sas_key=dummy_value")); + "https://testblobcontainer.blob.core.windows.net/?dummy_sas_key=dummy_value")); ASSERT_OK_AND_ASSIGN(options, AzureOptions::FromUri(uri)); ASSERT_EQ(options.credentials_kind, arrow::fs::AzureCredentialsKind::Sas); - ASSERT_EQ(options.account_dfs_url, "https://testcontainer.dfs.core.windows.net/"); - ASSERT_EQ(options.account_blob_url, "https://testcontainer.blob.core.windows.net/"); + ASSERT_EQ(options.account_dfs_url, "https://testblobcontainer.dfs.core.windows.net/"); + ASSERT_EQ(options.account_blob_url, "https://testblobcontainer.blob.core.windows.net/"); ASSERT_EQ(options.sas_token, "?dummy_sas_key=dummy_value"); } From 8600b6b3829f20bb9fdd5f07dd7dfc522ab0f016 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Sun, 28 Aug 2022 20:53:59 +0530 Subject: [PATCH 31/34] Updated versions.txt --- cpp/cmake_modules/ThirdpartyToolchain.cmake | 17 +++++++++-------- cpp/src/arrow/filesystem/azurefs.cc | 8 ++++---- cpp/thirdparty/versions.txt | 16 ++++++++-------- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index 3a555b0c4ba35..5432ee2798ea3 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -457,10 +457,10 @@ else() endif() if(DEFINED ENV{ARROW_AZURE_STORAGE_BLOB_URL}) - set(AZURE_STORAGE_BLOB_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_BLOB_URL}") + set(AZURE_STORAGE_BLOBS_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_BLOB_URL}") else() - set_urls(AZURE_STORAGE_BLOB_SOURCE_URL - "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-blobs_${ARROW_AZURE_STORAGE_BLOB_BUILD_VERSION}.tar.gz" + set_urls(AZURE_STORAGE_BLOBS_SOURCE_URL + "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-blobs_${ARROW_AZURE_STORAGE_BLOBS_BUILD_VERSION}.tar.gz" ) endif() @@ -4841,14 +4841,15 @@ macro(build_azuresdk) set(AZURESDK_COMMON_CMAKE_ARGS ${EP_COMMON_CMAKE_ARGS} + "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" + "-DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}" -DBUILD_SHARED_LIBS=OFF -DCMAKE_INSTALL_LIBDIR=${AZURESDK_LIB_DIR} + -DDISABLE_AZURE_CORE_OPENTELEMETRY=ON -DENABLE_TESTING=OFF -DENABLE_UNITY_BUILD=ON -DOPENSSL_ROOT_DIR=${OPENSSL_ROOT_HINT} - -DWARNINGS_AS_ERRORS=OFF - "-DCMAKE_INSTALL_PREFIX=${AZURESDK_PREFIX}" - "-DCMAKE_PREFIX_PATH=${AZURESDK_PREFIX}") + -DWARNINGS_AS_ERRORS=OFF) file(MAKE_DIRECTORY ${AZURESDK_INCLUDE_DIR}) @@ -4894,8 +4895,8 @@ macro(build_azuresdk) externalproject_add(azure_storage_blobs_ep ${EP_LOG_OPTIONS} INSTALL_DIR ${AZURESDK_PREFIX} - URL ${AZURE_STORAGE_BLOB_SOURCE_URL} - URL_HASH "SHA256=${ARROW_AZURE_STORAGE_BLOB_BUILD_SHA256_CHECKSUM}" + URL ${AZURE_STORAGE_BLOBS_SOURCE_URL} + URL_HASH "SHA256=${ARROW_AZURE_STORAGE_BLOBS_BUILD_SHA256_CHECKSUM}" CMAKE_ARGS ${AZURESDK_COMMON_CMAKE_ARGS} BUILD_BYPRODUCTS ${AZURE_STORAGE_BLOBS_STATIC_LIBRARY}) add_library(Azure::azure-storage-blobs STATIC IMPORTED) diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index c1712a4b1a64a..35a87b2951fab 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -113,12 +113,12 @@ Status AzureOptions::ConfigureAccountKeyCredentials(const std::string& account_n } Status AzureOptions::ConfigureConnectionStringCredentials( - const std::string& connection_string_uri) { + const std::string& connection_string_key) { ARROW_ASSIGN_OR_RAISE(auto account_name, - GetAccountNameFromConnectionString(connection_string_uri)); + GetAccountNameFromConnectionString(connection_string_key)); account_dfs_url = "https://" + account_name + ".dfs.core.windows.net/"; account_blob_url = "https://" + account_name + ".blob.core.windows.net/"; - connection_string = connection_string_uri; + connection_string = connection_string_key; credentials_kind = AzureCredentialsKind::ConnectionString; return Status::OK(); } @@ -1876,7 +1876,7 @@ Status AzureBlobFileSystem::CreateDir(const std::string& s, bool recursive) { } else { // Check parent dir exists if (path.has_parent()) { - AzurePath parent_path = path.parent(); + auto parent_path = path.parent(); if (parent_path.path_to_file.empty()) { ARROW_ASSIGN_OR_RAISE(auto exists, impl_->ContainerExists(parent_path.container)); if (!exists) { diff --git a/cpp/thirdparty/versions.txt b/cpp/thirdparty/versions.txt index a8797de94b214..3ecb7609161e7 100644 --- a/cpp/thirdparty/versions.txt +++ b/cpp/thirdparty/versions.txt @@ -33,14 +33,14 @@ ARROW_AWS_C_COMMON_BUILD_VERSION=v0.6.9 ARROW_AWS_C_COMMON_BUILD_SHA256_CHECKSUM=928a3e36f24d1ee46f9eec360ec5cebfe8b9b8994fe39d4fa74ff51aebb12717 ARROW_AWS_C_EVENT_STREAM_BUILD_VERSION=v0.1.5 ARROW_AWS_C_EVENT_STREAM_BUILD_SHA256_CHECKSUM=f1b423a487b5d6dca118bfc0d0c6cc596dc476b282258a3228e73a8f730422d4 -ARROW_AZURE_CORE_BUILD_VERSION=1.5.0 -ARROW_AZURE_CORE_BUILD_SHA256_CHECKSUM=dab2caa54d062b61dbe982e29a4f1fcc70216b51b038a807763712a40dd258e9 -ARROW_AZURE_IDENTITY_BUILD_VERSION=1.2.0 -ARROW_AZURE_IDENTITY_BUILD_SHA256_CHECKSUM=ad4702890c25f956c59a63be4571a08ae0690fa6d2bfbebf326d0fd2e9b72945 -ARROW_AZURE_STORAGE_BLOB_BUILD_VERSION=12.4.0 -ARROW_AZURE_STORAGE_BLOB_BUILD_SHA256_CHECKSUM=ce77055ff5e1b88826a89a29399ffbdcdc77beca1eae61c81f34a3f6e0a20715 -ARROW_AZURE_STORAGE_COMMON_BUILD_VERSION=12.2.3 -ARROW_AZURE_STORAGE_COMMON_BUILD_SHA256_CHECKSUM=2d58e9c314b1b32f7d09880239a4ecce6686ed6df236a58f681ae5d526ed6201 +ARROW_AZURE_CORE_BUILD_VERSION=1.7.1 +ARROW_AZURE_CORE_BUILD_SHA256_CHECKSUM=ae6f03e65d9773d11cf3b9619d0bc7f567272974cf31b9e1c8ca2fa0ea4fb4c6 +ARROW_AZURE_IDENTITY_BUILD_VERSION=1.3.0 +ARROW_AZURE_IDENTITY_BUILD_SHA256_CHECKSUM=46701acd8000f317d1c4b33263d5d3203924fadcfa5af4860ae9187046a72c45 +ARROW_AZURE_STORAGE_BLOBS_BUILD_VERSION=12.5.0 +ARROW_AZURE_STORAGE_BLOBS_BUILD_SHA256_CHECKSUM=12394d864144ced9fc3562ad48cfe3426604e871b5aa72853ca398e086f0c594 +ARROW_AZURE_STORAGE_COMMON_BUILD_VERSION=12.2.4 +ARROW_AZURE_STORAGE_COMMON_BUILD_SHA256_CHECKSUM=7644b4355b492ba2039236b9fd56c3e7bb80aad983d8bac6a731d74aaf64e03f ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_VERSION=12.3.1 ARROW_AZURE_STORAGE_FILES_DATALAKE_BUILD_SHA256_CHECKSUM=a5b74076a751d7cfaf7c56674a40ce2792c4fab9add18758fab1fe091d00baff ARROW_BOOST_BUILD_VERSION=1.75.0 From b5327019f1a1f5a05c961137401bf6cb46dd5d7f Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 29 Aug 2022 12:12:34 +0530 Subject: [PATCH 32/34] Fixed ARROW_AZURE_STORAGE_BLOBS_URL --- cpp/cmake_modules/ThirdpartyToolchain.cmake | 4 ++-- cpp/src/arrow/filesystem/azurefs.cc | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/cpp/cmake_modules/ThirdpartyToolchain.cmake b/cpp/cmake_modules/ThirdpartyToolchain.cmake index ee15b55e44bec..0b19c8d449626 100644 --- a/cpp/cmake_modules/ThirdpartyToolchain.cmake +++ b/cpp/cmake_modules/ThirdpartyToolchain.cmake @@ -478,8 +478,8 @@ else() ) endif() -if(DEFINED ENV{ARROW_AZURE_STORAGE_BLOB_URL}) - set(AZURE_STORAGE_BLOBS_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_BLOB_URL}") +if(DEFINED ENV{ARROW_AZURE_STORAGE_BLOBS_URL}) + set(AZURE_STORAGE_BLOBS_SOURCE_URL "$ENV{ARROW_AZURE_STORAGE_BLOBS_URL}") else() set_urls(AZURE_STORAGE_BLOBS_SOURCE_URL "https://github.com/Azure/azure-sdk-for-cpp/archive/azure-storage-blobs_${ARROW_AZURE_STORAGE_BLOBS_BUILD_VERSION}.tar.gz" diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 35a87b2951fab..2da39b6528fda 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -85,7 +85,8 @@ Result AzureOptions::GetAccountNameFromConnectionString( return Status::Invalid("Invalid Azure Blob Storage connection string: '", connection_string, "' passed"); } - std::string account_name = connection_string.substr(pos_text + text.size(), pos_semicolon); + std::string account_name = + connection_string.substr(pos_text + text.size(), pos_semicolon); return account_name; } @@ -256,7 +257,8 @@ struct AzurePath { // path_to_file = testdir/testfile.txt // path_to_file_parts = [testdir, testfile.txt] - // Expected input here => s = synapsemlfs/testdir/testfile.txt, http://127.0.0.1/accountName/pathToBlob + // Expected input here => s = synapsemlfs/testdir/testfile.txt, + // http://127.0.0.1/accountName/pathToBlob auto src = internal::RemoveTrailingSlash(s); if (src.starts_with("https://127.0.0.1") || src.starts_with("http://127.0.0.1")) { RETURN_NOT_OK(FromLocalHostString(&src)); From 200592b79c52e5ad57cd7c158d3382612dfa0195 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 29 Aug 2022 16:58:10 +0530 Subject: [PATCH 33/34] Added libxml2-dev --- ci/docker/ubuntu-18.04-cpp.dockerfile | 1 + ci/docker/ubuntu-20.04-cpp.dockerfile | 1 + ci/docker/ubuntu-22.04-cpp.dockerfile | 1 + 3 files changed, 3 insertions(+) diff --git a/ci/docker/ubuntu-18.04-cpp.dockerfile b/ci/docker/ubuntu-18.04-cpp.dockerfile index d8202a316c5a3..64ef1d34a3d24 100644 --- a/ci/docker/ubuntu-18.04-cpp.dockerfile +++ b/ci/docker/ubuntu-18.04-cpp.dockerfile @@ -79,6 +79,7 @@ RUN apt-get update -y -q && \ libre2-dev \ libsnappy-dev \ libssl-dev \ + libxml2-dev \ ninja-build \ pkg-config \ protobuf-compiler \ diff --git a/ci/docker/ubuntu-20.04-cpp.dockerfile b/ci/docker/ubuntu-20.04-cpp.dockerfile index ec21c8ae675ed..f50341aca095b 100644 --- a/ci/docker/ubuntu-20.04-cpp.dockerfile +++ b/ci/docker/ubuntu-20.04-cpp.dockerfile @@ -90,6 +90,7 @@ RUN apt-get update -y -q && \ libssl-dev \ libthrift-dev \ libutf8proc-dev \ + libxml2-dev \ libzstd-dev \ make \ ninja-build \ diff --git a/ci/docker/ubuntu-22.04-cpp.dockerfile b/ci/docker/ubuntu-22.04-cpp.dockerfile index 31c3e8e78bdd8..6afd4ade89bce 100644 --- a/ci/docker/ubuntu-22.04-cpp.dockerfile +++ b/ci/docker/ubuntu-22.04-cpp.dockerfile @@ -89,6 +89,7 @@ RUN apt-get update -y -q && \ libsqlite3-dev \ libthrift-dev \ libutf8proc-dev \ + libxml2-dev \ libzstd-dev \ make \ ninja-build \ From 3ea2d7fae20742baaf670b81dfabdd33fcad0258 Mon Sep 17 00:00:00 2001 From: shefali singh Date: Mon, 3 Oct 2022 01:55:05 +0530 Subject: [PATCH 34/34] Fixed build errors --- cpp/CMakeLists.txt | 2 +- cpp/src/arrow/filesystem/azurefs.cc | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 84aca489c99f7..aaac71c5fb10f 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -774,7 +774,7 @@ if(ARROW_WITH_OPENTELEMETRY) endif() if(ARROW_AZURE) - list(APPEND ARROW_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) + list(APPEND ARROW_SHARED_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) list(APPEND ARROW_STATIC_LINK_LIBS ${AZURESDK_LINK_LIBRARIES}) endif() diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc index 2da39b6528fda..485f8afcd9490 100644 --- a/cpp/src/arrow/filesystem/azurefs.cc +++ b/cpp/src/arrow/filesystem/azurefs.cc @@ -54,6 +54,7 @@ #include "arrow/util/key_value_metadata.h" #include "arrow/util/logging.h" #include "arrow/util/optional.h" +#include "arrow/util/string.h" #include "arrow/util/task_group.h" #include "arrow/util/thread_pool.h" @@ -260,13 +261,13 @@ struct AzurePath { // Expected input here => s = synapsemlfs/testdir/testfile.txt, // http://127.0.0.1/accountName/pathToBlob auto src = internal::RemoveTrailingSlash(s); - if (src.starts_with("https://127.0.0.1") || src.starts_with("http://127.0.0.1")) { + if (arrow::internal::StartsWith(src, "https://127.0.0.1") || arrow::internal::StartsWith(src, "http://127.0.0.1")) { RETURN_NOT_OK(FromLocalHostString(&src)); } auto input_path = std::string(src.data()); if (internal::IsLikelyUri(input_path)) { RETURN_NOT_OK(ExtractBlobPath(&input_path)); - src = util::string_view(input_path); + src = std::string_view(input_path); } src = internal::RemoveLeadingSlash(src); auto first_sep = src.find_first_of(kSep); @@ -285,7 +286,7 @@ struct AzurePath { return path; } - static Status FromLocalHostString(util::string_view* src) { + static Status FromLocalHostString(std::string_view* src) { // src = http://127.0.0.1:10000/accountName/pathToBlob Uri uri; RETURN_NOT_OK(uri.Parse(src->data()));