You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2021/12/29 18:30:06 UTC

[arrow] branch master updated: ARROW-15136: [C++] Make S3FS tests faster

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

lidavidm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new f5ab883  ARROW-15136: [C++] Make S3FS tests faster
f5ab883 is described below

commit f5ab8833867cb456190d656300cbbb2f7724563e
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Wed Dec 29 13:28:07 2021 -0500

    ARROW-15136: [C++] Make S3FS tests faster
    
    Manage a readhead queue of ready-to-use Minio servers to avoid waiting for process launch at the start of each test.
    
    Before (time for arrow-s3fs-test):
    ```
    real	0m59,841s
    user	0m13,669s
    sys	0m2,851s
    ```
    
    After:
    ```
    real	0m6,544s
    user	0m20,655s
    sys	0m4,328s
    ```
    
    Closes #12001 from pitrou/ARROW-15136-faster-s3fs-test
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: David Li <li...@gmail.com>
---
 cpp/cmake_modules/BuildUtils.cmake         |   9 +-
 cpp/src/arrow/filesystem/CMakeLists.txt    |  14 ++-
 cpp/src/arrow/filesystem/s3_test_util.cc   | 174 +++++++++++++++++++++++++++++
 cpp/src/arrow/filesystem/s3_test_util.h    | 107 ++++--------------
 cpp/src/arrow/filesystem/s3fs_benchmark.cc |   7 +-
 cpp/src/arrow/filesystem/s3fs_test.cc      |  85 ++++++++------
 6 files changed, 273 insertions(+), 123 deletions(-)

diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake
index 38c35d7..294b6ae 100644
--- a/cpp/cmake_modules/BuildUtils.cmake
+++ b/cpp/cmake_modules/BuildUtils.cmake
@@ -527,6 +527,7 @@ function(ADD_BENCHMARK REL_BENCHMARK_NAME)
       EXTRA_LINK_LIBS
       STATIC_LINK_LIBS
       DEPENDENCIES
+      SOURCES
       PREFIX
       LABELS)
   cmake_parse_arguments(ARG
@@ -547,13 +548,19 @@ function(ADD_BENCHMARK REL_BENCHMARK_NAME)
     set(BENCHMARK_NAME "${ARG_PREFIX}-${BENCHMARK_NAME}")
   endif()
 
+  if(ARG_SOURCES)
+    set(SOURCES ${ARG_SOURCES})
+  else()
+    set(SOURCES "${REL_BENCHMARK_NAME}.cc")
+  endif()
+
   # Make sure the executable name contains only hyphens, not underscores
   string(REPLACE "_" "-" BENCHMARK_NAME ${BENCHMARK_NAME})
 
   if(EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/${REL_BENCHMARK_NAME}.cc)
     # This benchmark has a corresponding .cc file, set it up as an executable.
     set(BENCHMARK_PATH "${EXECUTABLE_OUTPUT_PATH}/${BENCHMARK_NAME}")
-    add_executable(${BENCHMARK_NAME} "${REL_BENCHMARK_NAME}.cc")
+    add_executable(${BENCHMARK_NAME} ${SOURCES})
 
     if(ARG_STATIC_LINK_LIBS)
       # Customize link libraries
diff --git a/cpp/src/arrow/filesystem/CMakeLists.txt b/cpp/src/arrow/filesystem/CMakeLists.txt
index 67ebe54..c301a57 100644
--- a/cpp/src/arrow/filesystem/CMakeLists.txt
+++ b/cpp/src/arrow/filesystem/CMakeLists.txt
@@ -33,7 +33,12 @@ if(ARROW_GCS)
 endif()
 
 if(ARROW_S3)
-  add_arrow_test(s3fs_test EXTRA_LABELS filesystem)
+  add_arrow_test(s3fs_test
+                 SOURCES
+                 s3fs_test.cc
+                 s3_test_util.cc
+                 EXTRA_LABELS
+                 filesystem)
   if(TARGET arrow-s3fs-test)
     set(ARROW_S3FS_TEST_COMPILE_DEFINITIONS ${ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS})
     get_target_property(AWS_CPP_SDK_S3_TYPE aws-cpp-sdk-s3 TYPE)
@@ -63,7 +68,12 @@ if(ARROW_S3)
   endif()
 
   if(ARROW_BUILD_BENCHMARKS AND ARROW_PARQUET)
-    add_arrow_benchmark(s3fs_benchmark PREFIX "arrow-filesystem")
+    add_arrow_benchmark(s3fs_benchmark
+                        PREFIX
+                        "arrow-filesystem"
+                        SOURCES
+                        s3fs_benchmark.cc
+                        s3_test_util.cc)
     target_compile_definitions(arrow-filesystem-s3fs-benchmark
                                PRIVATE ${ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS})
     if(ARROW_TEST_LINKAGE STREQUAL "static")
diff --git a/cpp/src/arrow/filesystem/s3_test_util.cc b/cpp/src/arrow/filesystem/s3_test_util.cc
new file mode 100644
index 0000000..e47560f
--- /dev/null
+++ b/cpp/src/arrow/filesystem/s3_test_util.cc
@@ -0,0 +1,174 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <algorithm>  // Missing include in boost/process
+#include <sstream>
+
+// This boost/asio/io_context.hpp include is needless for no MinGW
+// build.
+//
+// This is for including boost/asio/detail/socket_types.hpp before any
+// "#include <windows.h>". boost/asio/detail/socket_types.hpp doesn't
+// work if windows.h is already included. boost/process.h ->
+// boost/process/args.hpp -> boost/process/detail/basic_cmd.hpp
+// includes windows.h. boost/process/args.hpp is included before
+// boost/process/async.h that includes
+// boost/asio/detail/socket_types.hpp implicitly is included.
+#include <boost/asio/io_context.hpp>
+// We need BOOST_USE_WINDOWS_H definition with MinGW when we use
+// boost/process.hpp. See ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS in
+// cpp/cmake_modules/BuildUtils.cmake for details.
+#include <boost/process.hpp>
+
+#include "arrow/filesystem/s3_test_util.h"
+#include "arrow/filesystem/s3fs.h"
+#include "arrow/util/async_generator.h"
+#include "arrow/util/future.h"
+#include "arrow/util/io_util.h"
+#include "arrow/util/thread_pool.h"
+
+namespace arrow {
+namespace fs {
+
+using ::arrow::internal::TemporaryDir;
+
+namespace bp = boost::process;
+
+namespace {
+
+const char* kMinioExecutableName = "minio";
+const char* kMinioAccessKey = "minio";
+const char* kMinioSecretKey = "miniopass";
+
+// Environment variables to configure another S3-compatible service
+const char* kEnvConnectString = "ARROW_TEST_S3_CONNECT_STRING";
+const char* kEnvAccessKey = "ARROW_TEST_S3_ACCESS_KEY";
+const char* kEnvSecretKey = "ARROW_TEST_S3_SECRET_KEY";
+
+std::string GenerateConnectString() {
+  std::stringstream ss;
+  ss << "127.0.0.1:" << GetListenPort();
+  return ss.str();
+}
+
+}  // namespace
+
+struct MinioTestServer::Impl {
+  std::unique_ptr<TemporaryDir> temp_dir_;
+  std::string connect_string_;
+  std::string access_key_ = kMinioAccessKey;
+  std::string secret_key_ = kMinioSecretKey;
+  std::shared_ptr<::boost::process::child> server_process_;
+};
+
+MinioTestServer::MinioTestServer() : impl_(new Impl) {}
+
+MinioTestServer::~MinioTestServer() {
+  auto st = Stop();
+  ARROW_UNUSED(st);
+}
+
+std::string MinioTestServer::connect_string() const { return impl_->connect_string_; }
+
+std::string MinioTestServer::access_key() const { return impl_->access_key_; }
+
+std::string MinioTestServer::secret_key() const { return impl_->secret_key_; }
+
+Status MinioTestServer::Start() {
+  const char* connect_str = std::getenv(kEnvConnectString);
+  const char* access_key = std::getenv(kEnvAccessKey);
+  const char* secret_key = std::getenv(kEnvSecretKey);
+  if (connect_str && access_key && secret_key) {
+    // Use external instance
+    impl_->connect_string_ = connect_str;
+    impl_->access_key_ = access_key;
+    impl_->secret_key_ = secret_key;
+    return Status::OK();
+  }
+
+  ARROW_ASSIGN_OR_RAISE(impl_->temp_dir_, TemporaryDir::Make("s3fs-test-"));
+
+  // Get a copy of the current environment.
+  // (NOTE: using "auto" would return a native_environment that mutates
+  //  the current environment)
+  bp::environment env = boost::this_process::environment();
+  env["MINIO_ACCESS_KEY"] = kMinioAccessKey;
+  env["MINIO_SECRET_KEY"] = kMinioSecretKey;
+
+  impl_->connect_string_ = GenerateConnectString();
+
+  auto exe_path = bp::search_path(kMinioExecutableName);
+  if (exe_path.empty()) {
+    return Status::IOError("Failed to find minio executable ('", kMinioExecutableName,
+                           "') in PATH");
+  }
+
+  try {
+    // NOTE: --quiet makes startup faster by suppressing remote version check
+    impl_->server_process_ = std::make_shared<bp::child>(
+        env, exe_path, "server", "--quiet", "--compat", "--address",
+        impl_->connect_string_, impl_->temp_dir_->path().ToString());
+  } catch (const std::exception& e) {
+    return Status::IOError("Failed to launch Minio server: ", e.what());
+  }
+  return Status::OK();
+}
+
+Status MinioTestServer::Stop() {
+  if (impl_->server_process_ && impl_->server_process_->valid()) {
+    // Brutal shutdown
+    impl_->server_process_->terminate();
+    impl_->server_process_->wait();
+  }
+  return Status::OK();
+}
+
+struct MinioTestEnvironment::Impl {
+  std::function<Future<std::shared_ptr<MinioTestServer>>()> server_generator_;
+
+  Result<std::shared_ptr<MinioTestServer>> LaunchOneServer() {
+    auto server = std::make_shared<MinioTestServer>();
+    RETURN_NOT_OK(server->Start());
+    return server;
+  }
+};
+
+MinioTestEnvironment::MinioTestEnvironment() : impl_(new Impl) {}
+
+MinioTestEnvironment::~MinioTestEnvironment() = default;
+
+void MinioTestEnvironment::SetUp() {
+  auto pool = ::arrow::internal::GetCpuThreadPool();
+
+  auto launch_one_server = []() -> Result<std::shared_ptr<MinioTestServer>> {
+    auto server = std::make_shared<MinioTestServer>();
+    RETURN_NOT_OK(server->Start());
+    return server;
+  };
+  impl_->server_generator_ = [pool, launch_one_server]() {
+    return DeferNotOk(pool->Submit(launch_one_server));
+  };
+  impl_->server_generator_ =
+      MakeReadaheadGenerator(std::move(impl_->server_generator_), pool->GetCapacity());
+}
+
+Result<std::shared_ptr<MinioTestServer>> MinioTestEnvironment::GetOneServer() {
+  return impl_->server_generator_().result();
+}
+
+}  // namespace fs
+}  // namespace arrow
diff --git a/cpp/src/arrow/filesystem/s3_test_util.h b/cpp/src/arrow/filesystem/s3_test_util.h
index 432ff1d..17245e0 100644
--- a/cpp/src/arrow/filesystem/s3_test_util.h
+++ b/cpp/src/arrow/filesystem/s3_test_util.h
@@ -18,116 +18,58 @@
 #pragma once
 
 #include <memory>
-#include <sstream>
 #include <string>
 #include <utility>
 
-// We need BOOST_USE_WINDOWS_H definition with MinGW when we use
-// boost/process.hpp. See ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS in
-// cpp/cmake_modules/BuildUtils.cmake for details.
-#include <aws/core/Aws.h>
 #include <gtest/gtest.h>
 
-#include <boost/process.hpp>
-
 #include "arrow/filesystem/s3fs.h"
 #include "arrow/status.h"
 #include "arrow/testing/gtest_util.h"
-#include "arrow/util/io_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/macros.h"
 
 namespace arrow {
 namespace fs {
 
-using ::arrow::internal::TemporaryDir;
-
-namespace bp = boost::process;
-
-// TODO: allocate an ephemeral port
-static const char* kMinioExecutableName = "minio";
-static const char* kMinioAccessKey = "minio";
-static const char* kMinioSecretKey = "miniopass";
-
-// Environment variables to configure another S3-compatible service
-static const char* kEnvConnectString = "ARROW_TEST_S3_CONNECT_STRING";
-static const char* kEnvAccessKey = "ARROW_TEST_S3_ACCESS_KEY";
-static const char* kEnvSecretKey = "ARROW_TEST_S3_SECRET_KEY";
-
-static std::string GenerateConnectString() {
-  std::stringstream ss;
-  ss << "127.0.0.1:" << GetListenPort();
-  return ss.str();
-}
-
 // A minio test server, managed as a child process
 
 class MinioTestServer {
  public:
+  MinioTestServer();
+  ~MinioTestServer();
+
   Status Start();
 
   Status Stop();
 
-  std::string connect_string() const { return connect_string_; }
+  std::string connect_string() const;
 
-  std::string access_key() const { return access_key_; }
+  std::string access_key() const;
 
-  std::string secret_key() const { return secret_key_; }
+  std::string secret_key() const;
 
  private:
-  std::unique_ptr<TemporaryDir> temp_dir_;
-  std::string connect_string_;
-  std::string access_key_ = kMinioAccessKey;
-  std::string secret_key_ = kMinioSecretKey;
-  std::shared_ptr<::boost::process::child> server_process_;
+  struct Impl;
+  std::unique_ptr<Impl> impl_;
 };
 
-Status MinioTestServer::Start() {
-  const char* connect_str = std::getenv(kEnvConnectString);
-  const char* access_key = std::getenv(kEnvAccessKey);
-  const char* secret_key = std::getenv(kEnvSecretKey);
-  if (connect_str && access_key && secret_key) {
-    // Use external instance
-    connect_string_ = connect_str;
-    access_key_ = access_key;
-    secret_key_ = secret_key;
-    return Status::OK();
-  }
-
-  ARROW_ASSIGN_OR_RAISE(temp_dir_, TemporaryDir::Make("s3fs-test-"));
+// A Minio "environment" that spawns Minio processes in advances, such as
+// to hide process launch latencies during testing.
 
-  // Get a copy of the current environment.
-  // (NOTE: using "auto" would return a native_environment that mutates
-  //  the current environment)
-  bp::environment env = boost::this_process::environment();
-  env["MINIO_ACCESS_KEY"] = kMinioAccessKey;
-  env["MINIO_SECRET_KEY"] = kMinioSecretKey;
+class MinioTestEnvironment : public ::testing::Environment {
+ public:
+  MinioTestEnvironment();
+  ~MinioTestEnvironment();
 
-  connect_string_ = GenerateConnectString();
+  void SetUp() override;
 
-  auto exe_path = bp::search_path(kMinioExecutableName);
-  if (exe_path.empty()) {
-    return Status::IOError("Failed to find minio executable ('", kMinioExecutableName,
-                           "') in PATH");
-  }
+  Result<std::shared_ptr<MinioTestServer>> GetOneServer();
 
-  try {
-    // NOTE: --quiet makes startup faster by suppressing remote version check
-    server_process_ = std::make_shared<bp::child>(
-        env, exe_path, "server", "--quiet", "--compat", "--address", connect_string_,
-        temp_dir_->path().ToString());
-  } catch (const std::exception& e) {
-    return Status::IOError("Failed to launch Minio server: ", e.what());
-  }
-  return Status::OK();
-}
-
-Status MinioTestServer::Stop() {
-  if (server_process_ && server_process_->valid()) {
-    // Brutal shutdown
-    server_process_->terminate();
-    server_process_->wait();
-  }
-  return Status::OK();
-}
+ protected:
+  struct Impl;
+  std::unique_ptr<Impl> impl_;
+};
 
 // A global test "environment", to ensure that the S3 API is initialized before
 // running unit tests.
@@ -142,12 +84,7 @@ class S3Environment : public ::testing::Environment {
   }
 
   void TearDown() override { ASSERT_OK(FinalizeS3()); }
-
- protected:
-  Aws::SDKOptions options_;
 };
 
-::testing::Environment* s3_env = ::testing::AddGlobalTestEnvironment(new S3Environment);
-
 }  // namespace fs
 }  // namespace arrow
diff --git a/cpp/src/arrow/filesystem/s3fs_benchmark.cc b/cpp/src/arrow/filesystem/s3fs_benchmark.cc
index 869601b..c664b76 100644
--- a/cpp/src/arrow/filesystem/s3fs_benchmark.cc
+++ b/cpp/src/arrow/filesystem/s3fs_benchmark.cc
@@ -195,8 +195,11 @@ class MinioFixture : public benchmark::Fixture {
 /// (GBenchmark doesn't run GTest environments.)
 class S3BenchmarkEnvironment {
  public:
-  S3BenchmarkEnvironment() { s3_env->SetUp(); }
-  ~S3BenchmarkEnvironment() { s3_env->TearDown(); }
+  S3BenchmarkEnvironment() { s3_env_.SetUp(); }
+  ~S3BenchmarkEnvironment() { s3_env_.TearDown(); }
+
+ private:
+  S3Environment s3_env_;
 };
 
 S3BenchmarkEnvironment env{};
diff --git a/cpp/src/arrow/filesystem/s3fs_test.cc b/cpp/src/arrow/filesystem/s3fs_test.cc
index db1249f..ea9cfe50 100644
--- a/cpp/src/arrow/filesystem/s3fs_test.cc
+++ b/cpp/src/arrow/filesystem/s3fs_test.cc
@@ -23,22 +23,6 @@
 #include <utility>
 #include <vector>
 
-// This boost/asio/io_context.hpp include is needless for no MinGW
-// build.
-//
-// This is for including boost/asio/detail/socket_types.hpp before any
-// "#include <windows.h>". boost/asio/detail/socket_types.hpp doesn't
-// work if windows.h is already included. boost/process.h ->
-// boost/process/args.hpp -> boost/process/detail/basic_cmd.hpp
-// includes windows.h. boost/process/args.hpp is included before
-// boost/process/async.h that includes
-// boost/asio/detail/socket_types.hpp implicitly is included.
-#include <boost/asio/io_context.hpp>
-// We need BOOST_USE_WINDOWS_H definition with MinGW when we use
-// boost/process.hpp. See ARROW_BOOST_PROCESS_COMPILE_DEFINITIONS in
-// cpp/cmake_modules/BuildUtils.cmake for details.
-#include <boost/process.hpp>
-
 #include <gmock/gmock-matchers.h>
 #include <gtest/gtest.h>
 
@@ -95,7 +79,41 @@ using ::arrow::fs::internal::ErrorToStatus;
 using ::arrow::fs::internal::OutcomeToStatus;
 using ::arrow::fs::internal::ToAwsString;
 
-namespace bp = boost::process;
+// Use "short" retry parameters to make tests faster
+static constexpr int32_t kRetryInterval = 50;      /* milliseconds */
+static constexpr int32_t kMaxRetryDuration = 6000; /* milliseconds */
+
+::testing::Environment* s3_env = ::testing::AddGlobalTestEnvironment(new S3Environment);
+
+::testing::Environment* minio_env =
+    ::testing::AddGlobalTestEnvironment(new MinioTestEnvironment);
+
+MinioTestEnvironment* GetMinioEnv() {
+  return ::arrow::internal::checked_cast<MinioTestEnvironment*>(minio_env);
+}
+
+class ShortRetryStrategy : public S3RetryStrategy {
+ public:
+  bool ShouldRetry(const AWSErrorDetail& error, int64_t attempted_retries) override {
+    if (error.message.find(kFileExistsMessage) != error.message.npos) {
+      // Minio returns "file exists" errors (when calling mkdir) as internal errors,
+      // which would trigger spurious retries.
+      return false;
+    }
+    return error.should_retry && (attempted_retries * kRetryInterval < kMaxRetryDuration);
+  }
+
+  int64_t CalculateDelayBeforeNextRetry(const AWSErrorDetail& error,
+                                        int64_t attempted_retries) override {
+    return kRetryInterval;
+  }
+
+#ifdef _WIN32
+  static constexpr const char* kFileExistsMessage = "file already exists";
+#else
+  static constexpr const char* kFileExistsMessage = "file exists";
+#endif
+};
 
 // NOTE: Connecting in Python:
 // >>> fs = s3fs.S3FileSystem(key='minio', secret='miniopass',
@@ -157,13 +175,14 @@ class S3TestMixin : public AwsTestMixin {
   void SetUp() override {
     AwsTestMixin::SetUp();
 
-    ASSERT_OK(minio_.Start());
+    ASSERT_OK_AND_ASSIGN(minio_, GetMinioEnv()->GetOneServer());
 
     client_config_.reset(new Aws::Client::ClientConfiguration());
-    client_config_->endpointOverride = ToAwsString(minio_.connect_string());
+    client_config_->endpointOverride = ToAwsString(minio_->connect_string());
     client_config_->scheme = Aws::Http::Scheme::HTTP;
-    client_config_->retryStrategy = std::make_shared<ConnectRetryStrategy>();
-    credentials_ = {ToAwsString(minio_.access_key()), ToAwsString(minio_.secret_key())};
+    client_config_->retryStrategy =
+        std::make_shared<ConnectRetryStrategy>(kRetryInterval, kMaxRetryDuration);
+    credentials_ = {ToAwsString(minio_->access_key()), ToAwsString(minio_->secret_key())};
     bool use_virtual_addressing = false;
     client_.reset(
         new Aws::S3::S3Client(credentials_, *client_config_,
@@ -171,14 +190,10 @@ class S3TestMixin : public AwsTestMixin {
                               use_virtual_addressing));
   }
 
-  void TearDown() override {
-    ASSERT_OK(minio_.Stop());
-
-    AwsTestMixin::TearDown();
-  }
+  void TearDown() override { AwsTestMixin::TearDown(); }
 
  protected:
-  MinioTestServer minio_;
+  std::shared_ptr<MinioTestServer> minio_;
   std::unique_ptr<Aws::Client::ClientConfiguration> client_config_;
   Aws::Auth::AWSCredentials credentials_;
   std::unique_ptr<Aws::S3::S3Client> client_;
@@ -416,9 +431,12 @@ class TestS3FS : public S3TestMixin {
   }
 
   void MakeFileSystem() {
-    options_.ConfigureAccessKey(minio_.access_key(), minio_.secret_key());
+    options_.ConfigureAccessKey(minio_->access_key(), minio_->secret_key());
     options_.scheme = "http";
-    options_.endpoint_override = minio_.connect_string();
+    options_.endpoint_override = minio_->connect_string();
+    if (!options_.retry_strategy) {
+      options_.retry_strategy = std::make_shared<ShortRetryStrategy>();
+    }
     ASSERT_OK_AND_ASSIGN(fs_, S3FileSystem::Make(options_));
   }
 
@@ -1045,9 +1063,9 @@ TEST_F(TestS3FS, OpenOutputStreamMetadata) {
 
 TEST_F(TestS3FS, FileSystemFromUri) {
   std::stringstream ss;
-  ss << "s3://" << minio_.access_key() << ":" << minio_.secret_key()
+  ss << "s3://" << minio_->access_key() << ":" << minio_->secret_key()
      << "@bucket/somedir/subdir/subfile"
-     << "?scheme=http&endpoint_override=" << UriEscape(minio_.connect_string());
+     << "?scheme=http&endpoint_override=" << UriEscape(minio_->connect_string());
 
   std::string path;
   ASSERT_OK_AND_ASSIGN(auto fs, FileSystemFromUri(ss.str(), &path));
@@ -1117,9 +1135,10 @@ class TestS3FSGeneric : public S3TestMixin, public GenericFileSystemTest {
       ASSERT_OK(OutcomeToStatus(client_->CreateBucket(req)));
     }
 
-    options_.ConfigureAccessKey(minio_.access_key(), minio_.secret_key());
+    options_.ConfigureAccessKey(minio_->access_key(), minio_->secret_key());
     options_.scheme = "http";
-    options_.endpoint_override = minio_.connect_string();
+    options_.endpoint_override = minio_->connect_string();
+    options_.retry_strategy = std::make_shared<ShortRetryStrategy>();
     ASSERT_OK_AND_ASSIGN(s3fs_, S3FileSystem::Make(options_));
     fs_ = std::make_shared<SubTreeFileSystem>("s3fs-test-bucket", s3fs_);
   }