wgtmac commented on code in PR #361:
URL: https://github.com/apache/iceberg-cpp/pull/361#discussion_r2579319386


##########
src/iceberg/test/meson.build:
##########
@@ -88,27 +86,39 @@ iceberg_tests = {
 }
 
 if get_option('rest').enabled()
-    cpp_httplib_dep = dependency('cpp-httplib')

Review Comment:
   Please also remove this file: `subprojects/cpp-httplib.wrap`



##########
src/iceberg/test/CMakeLists.txt:
##########
@@ -48,16 +39,15 @@ function(add_iceberg_test test_name)
                         ${ARGN})
 
   add_executable(${test_name})
-  target_include_directories(${test_name} PRIVATE 
"${CMAKE_BINARY_DIR}/iceberg/test/")
+  target_include_directories(${test_name} PRIVATE 
"${CMAKE_BINARY_DIR}/iceberg/test/"
+                                                  "${CMAKE_SOURCE_DIR}/src")

Review Comment:
   Why do we need to add `"${CMAKE_SOURCE_DIR}/src"` again?



##########
src/iceberg/test/CMakeLists.txt:
##########
@@ -48,16 +39,15 @@ function(add_iceberg_test test_name)
                         ${ARGN})
 
   add_executable(${test_name})
-  target_include_directories(${test_name} PRIVATE 
"${CMAKE_BINARY_DIR}/iceberg/test/")
+  target_include_directories(${test_name} PRIVATE 
"${CMAKE_BINARY_DIR}/iceberg/test/"
+                                                  "${CMAKE_SOURCE_DIR}/src")
 
-  target_sources(${test_name} PRIVATE ${ARG_SOURCES})
+  target_sources(${test_name} PRIVATE ${ARG_SOURCES} util/common_util.cc)

Review Comment:
   In general, it is not a good practice to mix unrelated changes in a single 
PR. I'll remove this file since it has been misused by different test cases.



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>

Review Comment:
   Move it closer to <sys/socket.h>?



##########
src/iceberg/test/resources/docker-compose.yml:
##########
@@ -0,0 +1,34 @@
+# 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.
+
+networks:
+  rest_bridge:
+
+services:
+  rest:
+    image: apache/iceberg-rest-fixture:1.10.0
+    environment:
+      - AWS_ACCESS_KEY_ID=admin
+      - AWS_SECRET_ACCESS_KEY=password
+      - AWS_REGION=us-east-1
+      - CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog
+      - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory
+      - CATALOG_WAREHOUSE=file:///tmp/iceberg_warehouse
+    networks:

Review Comment:
   Do we need this?



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>
+
+#include <chrono>
+#include <memory>
+#include <print>
 #include <string>
+#include <thread>
 #include <unordered_map>
 
+#include <arpa/inet.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
 
 #include "iceberg/catalog/rest/catalog_properties.h"
+#include "iceberg/result.h"
 #include "iceberg/table_identifier.h"
 #include "iceberg/test/matchers.h"
+#include "iceberg/test/util/docker_compose_util.h"
 
 namespace iceberg::rest {
 
-// Test fixture for REST catalog tests, This assumes you have a local REST 
catalog service
-// running Default configuration: http://localhost:8181.
-class RestCatalogTest : public ::testing::Test {
+namespace {
+
+constexpr uint16_t kRestCatalogPort = 8181;
+constexpr int kMaxRetries = 60;  // Wait up to 60 seconds
+constexpr int kRetryDelayMs = 1000;
+
+constexpr std::string_view kDockerProjectName = "iceberg-rest-catalog-service";
+constexpr std::string_view kCatalogName = "test_catalog";
+constexpr std::string_view kWarehouseName = "default";
+constexpr std::string_view kLocalhostUri = "http://localhost";;
+
+/// \brief Check if a localhost port is ready to accept connections
+/// \param port Port number to check
+/// \return true if the port is accessible on localhost, false otherwise
+bool CheckServiceReady(uint16_t port) {
+  int sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    return false;
+  }
+
+  struct timeval timeout{
+      .tv_sec = 1,
+      .tv_usec = 0,
+  };
+  setsockopt(sock, SOL_SOCKET, SO_RCVTIMEO, &timeout, sizeof(timeout));
+
+  sockaddr_in addr{
+      .sin_family = AF_INET,
+      .sin_port = htons(port),
+      .sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}  // 127.0.0.1
+  };
+  bool result =
+      (connect(sock, reinterpret_cast<struct sockaddr*>(&addr), sizeof(addr)) 
== 0);
+  close(sock);
+  return result;
+}
+
+}  // namespace
+
+/// \brief Integration test fixture for REST catalog with automatic Docker 
Compose setup。
+class RestCatalogIntegrationTest : public ::testing::Test {
  protected:
-  void SetUp() override {
-    // Default configuration for local testing
-    // You can override this with environment variables if needed
-    const char* uri_env = std::getenv("ICEBERG_REST_URI");
-    const char* warehouse_env = std::getenv("ICEBERG_REST_WAREHOUSE");
+  static void SetUpTestSuite() {
+    std::string project_name{kDockerProjectName};
+    std::filesystem::path resources_dir =
+        std::filesystem::path(__FILE__).parent_path() / "resources";
+
+    // Create and start DockerCompose
+    docker_compose_ = std::make_unique<DockerCompose>(project_name, 
resources_dir);
+    docker_compose_->Up();
+
+    // Wait for REST catalog to be ready on localhost
+    std::println("[INFO] Waiting for REST catalog to be ready at 
localhost:{}...",
+                 kRestCatalogPort);
+    for (int i = 0; i < kMaxRetries; ++i) {
+      if (CheckServiceReady(kRestCatalogPort)) {
+        std::println("[INFO] REST catalog is ready!");
+        return;
+      }
+      std::println(
+          "[INFO] Waiting for 1s for REST catalog to be ready... (attempt 
{}/{})", i + 1,
+          kMaxRetries);
+      std::this_thread::sleep_for(std::chrono::milliseconds(kRetryDelayMs));
+    }
+    throw RestError("REST catalog failed to start within {} seconds", 
kMaxRetries);
+  }
 
-    std::string uri = uri_env ? uri_env : "http://localhost:8181";;
-    std::string warehouse = warehouse_env ? warehouse_env : "default";
+  static void TearDownTestSuite() { docker_compose_.reset(); }
 
+  void SetUp() override {
     config_ = RestCatalogProperties::default_properties();
-    config_->Set(RestCatalogProperties::kUri, uri)
-        .Set(RestCatalogProperties::kName, std::string("test_catalog"))
-        .Set(RestCatalogProperties::kWarehouse, warehouse);
+    config_
+        ->Set(RestCatalogProperties::kUri,
+              std::format("{}:{}", kLocalhostUri, kRestCatalogPort))
+        .Set(RestCatalogProperties::kName, std::string(kCatalogName))
+        .Set(RestCatalogProperties::kWarehouse, std::string(kWarehouseName));
   }
 
   void TearDown() override {}
 
+  /// \brief Helper function to create a REST catalog instance
+  Result<std::unique_ptr<RestCatalog>> CreateCatalog() {
+    return RestCatalog::Make(*config_);
+  }
+
+  static inline std::unique_ptr<DockerCompose> docker_compose_;
   std::unique_ptr<RestCatalogProperties> config_;
 };
 
-TEST_F(RestCatalogTest, DISABLED_MakeCatalogSuccess) {
-  auto catalog_result = RestCatalog::Make(*config_);
-  EXPECT_THAT(catalog_result, IsOk());
+TEST_F(RestCatalogIntegrationTest, MakeCatalogSuccess) {
+  auto catalog_result = CreateCatalog();
+  ASSERT_THAT(catalog_result, IsOk());
 
-  if (catalog_result.has_value()) {
-    auto& catalog = catalog_result.value();
-    EXPECT_EQ(catalog->name(), "test_catalog");
-  }
+  auto& catalog = catalog_result.value();
+  EXPECT_EQ(catalog->name(), kCatalogName);
 }
 
-TEST_F(RestCatalogTest, DISABLED_MakeCatalogEmptyUri) {
-  auto invalid_config = RestCatalogProperties::default_properties();
-  invalid_config->Set(RestCatalogProperties::kUri, std::string(""));
+TEST_F(RestCatalogIntegrationTest, ListNamespaces) {
+  auto catalog_result = CreateCatalog();
+  ASSERT_THAT(catalog_result, IsOk());
+  auto& catalog = catalog_result.value();
 
-  auto catalog_result = RestCatalog::Make(*invalid_config);
-  EXPECT_THAT(catalog_result, IsError(ErrorKind::kInvalidArgument));
-  EXPECT_THAT(catalog_result, HasErrorMessage("uri"));
+  Namespace root{.levels = {}};
+  auto result = catalog->ListNamespaces(root);
+  EXPECT_THAT(result, IsOk());
 }
 
-TEST_F(RestCatalogTest, DISABLED_MakeCatalogWithCustomProperties) {
-  auto custom_config = RestCatalogProperties::default_properties();
-  custom_config
-      ->Set(RestCatalogProperties::kUri, 
config_->Get(RestCatalogProperties::kUri))
-      .Set(RestCatalogProperties::kName, 
config_->Get(RestCatalogProperties::kName))
-      .Set(RestCatalogProperties::kWarehouse,
-           config_->Get(RestCatalogProperties::kWarehouse))
-      .Set(RestCatalogProperties::Entry<std::string>{"custom_prop", ""},
-           std::string("custom_value"))
-      .Set(RestCatalogProperties::Entry<std::string>{"timeout", ""},
-           std::string("30000"));
-
-  auto catalog_result = RestCatalog::Make(*custom_config);
-  EXPECT_THAT(catalog_result, IsOk());
+TEST_F(RestCatalogIntegrationTest, DISABLED_GetNonExistentNamespace) {

Review Comment:
   I'd suggest removing all test cases that are not ready to run since they are 
meaningless and confusing to maintain.



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>
+
+#include <chrono>
+#include <memory>
+#include <print>
 #include <string>
+#include <thread>
 #include <unordered_map>
 
+#include <arpa/inet.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
 
 #include "iceberg/catalog/rest/catalog_properties.h"
+#include "iceberg/result.h"
 #include "iceberg/table_identifier.h"
 #include "iceberg/test/matchers.h"
+#include "iceberg/test/util/docker_compose_util.h"
 
 namespace iceberg::rest {
 
-// Test fixture for REST catalog tests, This assumes you have a local REST 
catalog service
-// running Default configuration: http://localhost:8181.
-class RestCatalogTest : public ::testing::Test {
+namespace {
+
+constexpr uint16_t kRestCatalogPort = 8181;
+constexpr int kMaxRetries = 60;  // Wait up to 60 seconds
+constexpr int kRetryDelayMs = 1000;
+
+constexpr std::string_view kDockerProjectName = "iceberg-rest-catalog-service";
+constexpr std::string_view kCatalogName = "test_catalog";
+constexpr std::string_view kWarehouseName = "default";
+constexpr std::string_view kLocalhostUri = "http://localhost";;
+
+/// \brief Check if a localhost port is ready to accept connections
+/// \param port Port number to check
+/// \return true if the port is accessible on localhost, false otherwise
+bool CheckServiceReady(uint16_t port) {
+  int sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    return false;
+  }
+
+  struct timeval timeout{
+      .tv_sec = 1,
+      .tv_usec = 0,
+  };
+  setsockopt(sock, SOL_SOCKET, SO_RCVTIMEO, &timeout, sizeof(timeout));
+
+  sockaddr_in addr{
+      .sin_family = AF_INET,
+      .sin_port = htons(port),
+      .sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}  // 127.0.0.1
+  };
+  bool result =
+      (connect(sock, reinterpret_cast<struct sockaddr*>(&addr), sizeof(addr)) 
== 0);
+  close(sock);
+  return result;
+}
+
+}  // namespace
+
+/// \brief Integration test fixture for REST catalog with automatic Docker 
Compose setup。
+class RestCatalogIntegrationTest : public ::testing::Test {
  protected:
-  void SetUp() override {
-    // Default configuration for local testing
-    // You can override this with environment variables if needed
-    const char* uri_env = std::getenv("ICEBERG_REST_URI");
-    const char* warehouse_env = std::getenv("ICEBERG_REST_WAREHOUSE");
+  static void SetUpTestSuite() {
+    std::string project_name{kDockerProjectName};
+    std::filesystem::path resources_dir =
+        std::filesystem::path(__FILE__).parent_path() / "resources";
+
+    // Create and start DockerCompose
+    docker_compose_ = std::make_unique<DockerCompose>(project_name, 
resources_dir);
+    docker_compose_->Up();
+
+    // Wait for REST catalog to be ready on localhost
+    std::println("[INFO] Waiting for REST catalog to be ready at 
localhost:{}...",
+                 kRestCatalogPort);
+    for (int i = 0; i < kMaxRetries; ++i) {
+      if (CheckServiceReady(kRestCatalogPort)) {
+        std::println("[INFO] REST catalog is ready!");
+        return;
+      }
+      std::println(
+          "[INFO] Waiting for 1s for REST catalog to be ready... (attempt 
{}/{})", i + 1,
+          kMaxRetries);
+      std::this_thread::sleep_for(std::chrono::milliseconds(kRetryDelayMs));
+    }
+    throw RestError("REST catalog failed to start within {} seconds", 
kMaxRetries);
+  }
 
-    std::string uri = uri_env ? uri_env : "http://localhost:8181";;
-    std::string warehouse = warehouse_env ? warehouse_env : "default";
+  static void TearDownTestSuite() { docker_compose_.reset(); }
 
+  void SetUp() override {
     config_ = RestCatalogProperties::default_properties();
-    config_->Set(RestCatalogProperties::kUri, uri)
-        .Set(RestCatalogProperties::kName, std::string("test_catalog"))
-        .Set(RestCatalogProperties::kWarehouse, warehouse);
+    config_

Review Comment:
   It seems that we don't need to keep the config instance since it is a 
one-off thing. You may directly create a config in the `CreateCatalog()` and 
throw it away after creating the catalog.



##########
src/iceberg/test/util/cmd_util.h:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <map>
+#include <string>
+#include <vector>
+
+/// \file iceberg/test/util/cmd_util.h
+/// Utilities for building and executing shell commands in tests.
+
+namespace iceberg {
+
+/// \brief A shell command builder and executor for tests.
+class Command {
+ public:
+  explicit Command(std::string program);
+
+  /// \brief Add a single argument
+  Command& Arg(std::string a);
+
+  /// \brief Add multiple arguments at once
+  Command& Args(const std::vector<std::string>& as);
+
+  /// \brief Set the current working directory for the command
+  Command& CurrentDir(const std::filesystem::path& path);
+
+  /// \brief Set an environment variable for the command
+  Command& Env(const std::string& key, const std::string& val);
+
+  /// \brief Execute the command and print logs
+  /// \return A Status indicating success or failure
+  void RunCommand(const std::string& desc) const;
+
+ private:
+  std::string program_;
+  std::vector<std::string> args_;
+  std::filesystem::path cwd_;
+  std::map<std::string, std::string> env_vars_;
+
+  /// \brief Format arguments for logging
+  std::string fmt_args() const;

Review Comment:
   ```suggestion
     std::string FormatArgs() const;
   ```



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>
+
+#include <chrono>
+#include <memory>
+#include <print>
 #include <string>
+#include <thread>
 #include <unordered_map>
 
+#include <arpa/inet.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
 
 #include "iceberg/catalog/rest/catalog_properties.h"
+#include "iceberg/result.h"
 #include "iceberg/table_identifier.h"
 #include "iceberg/test/matchers.h"
+#include "iceberg/test/util/docker_compose_util.h"
 
 namespace iceberg::rest {
 
-// Test fixture for REST catalog tests, This assumes you have a local REST 
catalog service
-// running Default configuration: http://localhost:8181.
-class RestCatalogTest : public ::testing::Test {
+namespace {
+
+constexpr uint16_t kRestCatalogPort = 8181;
+constexpr int kMaxRetries = 60;  // Wait up to 60 seconds
+constexpr int kRetryDelayMs = 1000;
+
+constexpr std::string_view kDockerProjectName = "iceberg-rest-catalog-service";
+constexpr std::string_view kCatalogName = "test_catalog";
+constexpr std::string_view kWarehouseName = "default";
+constexpr std::string_view kLocalhostUri = "http://localhost";;
+
+/// \brief Check if a localhost port is ready to accept connections
+/// \param port Port number to check
+/// \return true if the port is accessible on localhost, false otherwise
+bool CheckServiceReady(uint16_t port) {
+  int sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    return false;
+  }
+
+  struct timeval timeout{
+      .tv_sec = 1,
+      .tv_usec = 0,
+  };
+  setsockopt(sock, SOL_SOCKET, SO_RCVTIMEO, &timeout, sizeof(timeout));
+
+  sockaddr_in addr{
+      .sin_family = AF_INET,
+      .sin_port = htons(port),
+      .sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}  // 127.0.0.1
+  };
+  bool result =
+      (connect(sock, reinterpret_cast<struct sockaddr*>(&addr), sizeof(addr)) 
== 0);
+  close(sock);
+  return result;
+}
+
+}  // namespace
+
+/// \brief Integration test fixture for REST catalog with automatic Docker 
Compose setup。
+class RestCatalogIntegrationTest : public ::testing::Test {
  protected:
-  void SetUp() override {
-    // Default configuration for local testing
-    // You can override this with environment variables if needed
-    const char* uri_env = std::getenv("ICEBERG_REST_URI");
-    const char* warehouse_env = std::getenv("ICEBERG_REST_WAREHOUSE");
+  static void SetUpTestSuite() {
+    std::string project_name{kDockerProjectName};
+    std::filesystem::path resources_dir =

Review Comment:
   It is better to follow `GetResourcePath` function to obtain this.



##########
src/iceberg/test/meson.build:
##########
@@ -88,27 +86,39 @@ iceberg_tests = {
 }
 
 if get_option('rest').enabled()
-    cpp_httplib_dep = dependency('cpp-httplib')
     iceberg_tests += {
         'rest_catalog_test': {
-            'sources': files(
-                'rest_catalog_test.cc',
-                'rest_json_internal_test.cc',
-                'rest_util_test.cc',
-            ),
-            'dependencies': [iceberg_rest_dep, cpp_httplib_dep],
+            'sources': files('rest_json_internal_test.cc', 
'rest_util_test.cc'),
+            'dependencies': [iceberg_rest_dep],
         },
     }
+    if get_option('rest_integration_test').enabled()
+        if host_machine.system() == 'windows'
+            warning('Cannot build rest integration test on Windows, skipping.')
+        else
+            iceberg_tests += {
+                'rest_integration_test': {
+                    'sources': files(
+                        'rest_catalog_test.cc',
+                        'util/cmd_util.cc',
+                        'util/docker_compose_util.cc',
+                    ),
+                    'dependencies': [iceberg_rest_dep],
+                },
+            }
+        endif
+    endif
 endif
 
 foreach test_name, values : iceberg_tests
     exc = executable(
         test_name,
-        sources: values['sources'],
+        sources: values['sources'] + files('util/common_util.cc'),
         dependencies: [iceberg_dep, gmock_main_dep] + values.get(
             'dependencies',
             [],
         ),
+        include_directories: include_directories('.'),

Review Comment:
   Why do we need this? It seems unnecessary.



##########
src/iceberg/test/resources/docker-compose.yml:
##########
@@ -0,0 +1,34 @@
+# 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.
+
+networks:
+  rest_bridge:
+
+services:
+  rest:
+    image: apache/iceberg-rest-fixture:1.10.0
+    environment:
+      - AWS_ACCESS_KEY_ID=admin

Review Comment:
   Do we need AWS configs?



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>
+
+#include <chrono>
+#include <memory>
+#include <print>
 #include <string>
+#include <thread>
 #include <unordered_map>
 
+#include <arpa/inet.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
 
 #include "iceberg/catalog/rest/catalog_properties.h"
+#include "iceberg/result.h"
 #include "iceberg/table_identifier.h"
 #include "iceberg/test/matchers.h"
+#include "iceberg/test/util/docker_compose_util.h"
 
 namespace iceberg::rest {
 
-// Test fixture for REST catalog tests, This assumes you have a local REST 
catalog service
-// running Default configuration: http://localhost:8181.
-class RestCatalogTest : public ::testing::Test {
+namespace {
+
+constexpr uint16_t kRestCatalogPort = 8181;
+constexpr int kMaxRetries = 60;  // Wait up to 60 seconds
+constexpr int kRetryDelayMs = 1000;
+
+constexpr std::string_view kDockerProjectName = "iceberg-rest-catalog-service";
+constexpr std::string_view kCatalogName = "test_catalog";
+constexpr std::string_view kWarehouseName = "default";
+constexpr std::string_view kLocalhostUri = "http://localhost";;
+
+/// \brief Check if a localhost port is ready to accept connections
+/// \param port Port number to check
+/// \return true if the port is accessible on localhost, false otherwise
+bool CheckServiceReady(uint16_t port) {
+  int sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    return false;
+  }
+
+  struct timeval timeout{
+      .tv_sec = 1,
+      .tv_usec = 0,
+  };
+  setsockopt(sock, SOL_SOCKET, SO_RCVTIMEO, &timeout, sizeof(timeout));
+
+  sockaddr_in addr{
+      .sin_family = AF_INET,
+      .sin_port = htons(port),
+      .sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}  // 127.0.0.1
+  };
+  bool result =
+      (connect(sock, reinterpret_cast<struct sockaddr*>(&addr), sizeof(addr)) 
== 0);
+  close(sock);
+  return result;
+}
+
+}  // namespace
+
+/// \brief Integration test fixture for REST catalog with automatic Docker 
Compose setup。
+class RestCatalogIntegrationTest : public ::testing::Test {
  protected:
-  void SetUp() override {
-    // Default configuration for local testing
-    // You can override this with environment variables if needed
-    const char* uri_env = std::getenv("ICEBERG_REST_URI");
-    const char* warehouse_env = std::getenv("ICEBERG_REST_WAREHOUSE");
+  static void SetUpTestSuite() {
+    std::string project_name{kDockerProjectName};
+    std::filesystem::path resources_dir =
+        std::filesystem::path(__FILE__).parent_path() / "resources";
+
+    // Create and start DockerCompose
+    docker_compose_ = std::make_unique<DockerCompose>(project_name, 
resources_dir);
+    docker_compose_->Up();
+
+    // Wait for REST catalog to be ready on localhost
+    std::println("[INFO] Waiting for REST catalog to be ready at 
localhost:{}...",
+                 kRestCatalogPort);
+    for (int i = 0; i < kMaxRetries; ++i) {
+      if (CheckServiceReady(kRestCatalogPort)) {
+        std::println("[INFO] REST catalog is ready!");
+        return;
+      }
+      std::println(
+          "[INFO] Waiting for 1s for REST catalog to be ready... (attempt 
{}/{})", i + 1,
+          kMaxRetries);
+      std::this_thread::sleep_for(std::chrono::milliseconds(kRetryDelayMs));
+    }
+    throw RestError("REST catalog failed to start within {} seconds", 
kMaxRetries);

Review Comment:
   This is wrong.



##########
src/iceberg/test/rest_catalog_test.cc:
##########
@@ -19,103 +19,206 @@
 
 #include "iceberg/catalog/rest/rest_catalog.h"
 
+#include <unistd.h>
+
+#include <chrono>
+#include <memory>
+#include <print>
 #include <string>
+#include <thread>
 #include <unordered_map>
 
+#include <arpa/inet.h>
 #include <gmock/gmock.h>
 #include <gtest/gtest.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
 
 #include "iceberg/catalog/rest/catalog_properties.h"
+#include "iceberg/result.h"
 #include "iceberg/table_identifier.h"
 #include "iceberg/test/matchers.h"
+#include "iceberg/test/util/docker_compose_util.h"
 
 namespace iceberg::rest {
 
-// Test fixture for REST catalog tests, This assumes you have a local REST 
catalog service
-// running Default configuration: http://localhost:8181.
-class RestCatalogTest : public ::testing::Test {
+namespace {
+
+constexpr uint16_t kRestCatalogPort = 8181;
+constexpr int kMaxRetries = 60;  // Wait up to 60 seconds
+constexpr int kRetryDelayMs = 1000;
+
+constexpr std::string_view kDockerProjectName = "iceberg-rest-catalog-service";
+constexpr std::string_view kCatalogName = "test_catalog";
+constexpr std::string_view kWarehouseName = "default";
+constexpr std::string_view kLocalhostUri = "http://localhost";;
+
+/// \brief Check if a localhost port is ready to accept connections
+/// \param port Port number to check
+/// \return true if the port is accessible on localhost, false otherwise
+bool CheckServiceReady(uint16_t port) {
+  int sock = socket(AF_INET, SOCK_STREAM, 0);
+  if (sock < 0) {
+    return false;
+  }
+
+  struct timeval timeout{
+      .tv_sec = 1,
+      .tv_usec = 0,
+  };
+  setsockopt(sock, SOL_SOCKET, SO_RCVTIMEO, &timeout, sizeof(timeout));
+
+  sockaddr_in addr{
+      .sin_family = AF_INET,
+      .sin_port = htons(port),
+      .sin_addr = {.s_addr = htonl(INADDR_LOOPBACK)}  // 127.0.0.1
+  };
+  bool result =
+      (connect(sock, reinterpret_cast<struct sockaddr*>(&addr), sizeof(addr)) 
== 0);
+  close(sock);
+  return result;
+}
+
+}  // namespace
+
+/// \brief Integration test fixture for REST catalog with automatic Docker 
Compose setup。
+class RestCatalogIntegrationTest : public ::testing::Test {
  protected:
-  void SetUp() override {
-    // Default configuration for local testing
-    // You can override this with environment variables if needed
-    const char* uri_env = std::getenv("ICEBERG_REST_URI");
-    const char* warehouse_env = std::getenv("ICEBERG_REST_WAREHOUSE");
+  static void SetUpTestSuite() {
+    std::string project_name{kDockerProjectName};
+    std::filesystem::path resources_dir =
+        std::filesystem::path(__FILE__).parent_path() / "resources";
+
+    // Create and start DockerCompose
+    docker_compose_ = std::make_unique<DockerCompose>(project_name, 
resources_dir);
+    docker_compose_->Up();
+
+    // Wait for REST catalog to be ready on localhost
+    std::println("[INFO] Waiting for REST catalog to be ready at 
localhost:{}...",
+                 kRestCatalogPort);
+    for (int i = 0; i < kMaxRetries; ++i) {
+      if (CheckServiceReady(kRestCatalogPort)) {
+        std::println("[INFO] REST catalog is ready!");
+        return;
+      }
+      std::println(
+          "[INFO] Waiting for 1s for REST catalog to be ready... (attempt 
{}/{})", i + 1,
+          kMaxRetries);
+      std::this_thread::sleep_for(std::chrono::milliseconds(kRetryDelayMs));
+    }
+    throw RestError("REST catalog failed to start within {} seconds", 
kMaxRetries);
+  }
 
-    std::string uri = uri_env ? uri_env : "http://localhost:8181";;
-    std::string warehouse = warehouse_env ? warehouse_env : "default";
+  static void TearDownTestSuite() { docker_compose_.reset(); }
 
+  void SetUp() override {
     config_ = RestCatalogProperties::default_properties();
-    config_->Set(RestCatalogProperties::kUri, uri)
-        .Set(RestCatalogProperties::kName, std::string("test_catalog"))
-        .Set(RestCatalogProperties::kWarehouse, warehouse);
+    config_
+        ->Set(RestCatalogProperties::kUri,
+              std::format("{}:{}", kLocalhostUri, kRestCatalogPort))
+        .Set(RestCatalogProperties::kName, std::string(kCatalogName))
+        .Set(RestCatalogProperties::kWarehouse, std::string(kWarehouseName));
   }
 
   void TearDown() override {}
 
+  /// \brief Helper function to create a REST catalog instance
+  Result<std::unique_ptr<RestCatalog>> CreateCatalog() {
+    return RestCatalog::Make(*config_);
+  }
+
+  static inline std::unique_ptr<DockerCompose> docker_compose_;
   std::unique_ptr<RestCatalogProperties> config_;
 };
 
-TEST_F(RestCatalogTest, DISABLED_MakeCatalogSuccess) {
-  auto catalog_result = RestCatalog::Make(*config_);
-  EXPECT_THAT(catalog_result, IsOk());
+TEST_F(RestCatalogIntegrationTest, MakeCatalogSuccess) {
+  auto catalog_result = CreateCatalog();
+  ASSERT_THAT(catalog_result, IsOk());
 
-  if (catalog_result.has_value()) {
-    auto& catalog = catalog_result.value();
-    EXPECT_EQ(catalog->name(), "test_catalog");
-  }
+  auto& catalog = catalog_result.value();
+  EXPECT_EQ(catalog->name(), kCatalogName);
 }
 
-TEST_F(RestCatalogTest, DISABLED_MakeCatalogEmptyUri) {
-  auto invalid_config = RestCatalogProperties::default_properties();
-  invalid_config->Set(RestCatalogProperties::kUri, std::string(""));
+TEST_F(RestCatalogIntegrationTest, ListNamespaces) {
+  auto catalog_result = CreateCatalog();
+  ASSERT_THAT(catalog_result, IsOk());
+  auto& catalog = catalog_result.value();
 
-  auto catalog_result = RestCatalog::Make(*invalid_config);
-  EXPECT_THAT(catalog_result, IsError(ErrorKind::kInvalidArgument));
-  EXPECT_THAT(catalog_result, HasErrorMessage("uri"));
+  Namespace root{.levels = {}};
+  auto result = catalog->ListNamespaces(root);
+  EXPECT_THAT(result, IsOk());

Review Comment:
   Check the list returned is empty.



##########
src/iceberg/test/util/cmd_util.cc:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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 "iceberg/test/util/cmd_util.h"
+
+#include <unistd.h>

Review Comment:
   ditto, put all similar headers together



##########
src/iceberg/test/util/cmd_util.h:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <map>
+#include <string>
+#include <vector>
+
+/// \file iceberg/test/util/cmd_util.h
+/// Utilities for building and executing shell commands in tests.
+
+namespace iceberg {
+
+/// \brief A shell command builder and executor for tests.
+class Command {
+ public:
+  explicit Command(std::string program);
+
+  /// \brief Add a single argument
+  Command& Arg(std::string a);
+
+  /// \brief Add multiple arguments at once
+  Command& Args(const std::vector<std::string>& as);

Review Comment:
   ```suggestion
     Command& Args(const std::vector<std::string>& args);
   ```



##########
src/iceberg/test/util/cmd_util.h:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+#include <filesystem>
+#include <map>
+#include <string>
+#include <vector>
+
+/// \file iceberg/test/util/cmd_util.h
+/// Utilities for building and executing shell commands in tests.
+
+namespace iceberg {
+
+/// \brief A shell command builder and executor for tests.
+class Command {
+ public:
+  explicit Command(std::string program);
+
+  /// \brief Add a single argument
+  Command& Arg(std::string a);

Review Comment:
   ```suggestion
     Command& Arg(std::string arg);
   ```



##########
src/iceberg/test/resources/docker-compose.yml:
##########
@@ -0,0 +1,34 @@
+# 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.

Review Comment:
   What about putting this yaml file in a subdirectory like 
`resources/iceberg-rest-fixture`?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to