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


##########
src/iceberg/table_metadata.cc:
##########
@@ -668,6 +668,9 @@ class TableMetadataBuilder::Impl {
   Result<std::vector<SnapshotLogEntry>> UpdateSnapshotLog(
       int64_t current_snapshot_id) const;
 
+  /// \brief Internal method to set a branch snapshot

Review Comment:
   nit: internal (private) function does not need comment, especially 
self-descriptive functions like this.



##########
src/iceberg/transaction.cc:
##########
@@ -183,6 +185,12 @@ Status Transaction::Apply(PendingUpdate& update) {
         
metadata_builder_->RemoveSchemas(std::move(result.schema_ids_to_remove));
       }
     } break;
+    case PendingUpdate::Kind::kSetSnapshot: {
+      auto& set_snapshot = internal::checked_cast<SetSnapshot&>(update);
+      ICEBERG_ASSIGN_OR_RAISE(auto snapshot, set_snapshot.Apply());

Review Comment:
   If only `snapshot_id` is required here, why not just return `int64_t` from 
`Apply`?



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));

Review Comment:
   This is wrong. `*transaction_->table()` is out of date if the transaction 
has multiple updates. You need to use the current metadata to check the current 
snapshot.



##########
src/iceberg/test/set_snapshot_test.cc:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <memory>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/test/matchers.h"
+#include "iceberg/test/update_test_base.h"
+#include "iceberg/transaction.h"
+
+namespace iceberg {
+
+// Test fixture for SetSnapshot tests
+class SetSnapshotTest : public UpdateTestBase {
+ protected:
+  // Snapshot IDs from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotId = 3051729675574597004;
+  static constexpr int64_t kCurrentSnapshotId = 3055729675574597004;
+
+  // Timestamps from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotTimestamp = 1515100955770;
+  static constexpr int64_t kCurrentSnapshotTimestamp = 1555100955770;
+};
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the older snapshot
+  set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotToCurrentSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the current snapshot (no-op)
+  set_snapshot->SetCurrentSnapshot(kCurrentSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotInvalid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to set to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->SetCurrentSnapshot(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to the oldest snapshot (which is an ancestor)
+  set_snapshot->RollbackTo(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToInvalidSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->RollbackTo(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeValidOldestSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a time between the two snapshots
+  // This should select the oldest snapshot
+  int64_t time_between = (kOldestSnapshotTimestamp + 
kCurrentSnapshotTimestamp) / 2;
+  set_snapshot->RollbackToTime(time_between);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeBeforeAnySnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a time before any snapshot
+  int64_t time_before_all = kOldestSnapshotTimestamp - 1000000;
+  set_snapshot->RollbackToTime(time_before_all);
+
+  // Should fail - no snapshot older than the specified time
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("no valid snapshot older than"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeExactMatch) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a timestamp just after the oldest snapshot
+  // This should return the oldest snapshot (the latest one before this time)
+  int64_t time_just_after_oldest = kOldestSnapshotTimestamp + 1;
+  set_snapshot->RollbackToTime(time_just_after_oldest);
+
+  // Apply and verify - should return the oldest snapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, ApplyWithoutChanges) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Apply without making any changes (NOOP)
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+
+  // Should return current snapshot
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, MethodChaining) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Test that methods return reference for chaining
+  // Note: Only the last operation should take effect
+  auto& result1 = set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+  EXPECT_EQ(&result1, set_snapshot.get());
+}
+
+TEST_F(SetSnapshotTest, CommitSuccess) {

Review Comment:
   Can we remove `CommitXXX` test by calling commit in all above cases? They 
are too many tiny test cases here.



##########
src/iceberg/transaction.cc:
##########
@@ -183,6 +185,12 @@ Status Transaction::Apply(PendingUpdate& update) {
         
metadata_builder_->RemoveSchemas(std::move(result.schema_ids_to_remove));
       }
     } break;
+    case PendingUpdate::Kind::kSetSnapshot: {

Review Comment:
   We are missing a method from `Transaction` to create this instance. 
Otherwise this update is useless.



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);

Review Comment:
   ```suggestion
     ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot,
                                      base().SnapshotById(snapshot_id));
     ICEBERG_BUILDER_CHECK(snapshot != nullptr,
                           "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
   ```



##########
src/iceberg/update/set_snapshot.h:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+
+/// \file iceberg/update/set_snapshot.h
+/// \brief Sets the current snapshot directly or by rolling back.
+
+namespace iceberg {
+
+/// \brief Sets the current snapshot directly or by rolling back.
+class ICEBERG_EXPORT SetSnapshot : public PendingUpdate {
+ public:
+  static Result<std::shared_ptr<SetSnapshot>> Make(
+      std::shared_ptr<Transaction> transaction);
+
+  ~SetSnapshot() override;
+
+  /// \brief Sets the table's current state to a specific Snapshot identified 
by id.
+  ///
+  /// This method allows setting the current snapshot to <b>any</b> valid 
snapshot defined
+  /// in the table metadata, regardless of its relationship to the current 
state.
+  ///
+  /// \param snapshot_id The snapshot ID to set as current
+  /// \return Reference to this for method chaining

Review Comment:
   ```suggestion
     /// \brief Sets the table's current state to a specific Snapshot 
identified by id.
   ```
   
   Currently LLM is notorious at generating too verbose comments like this. 
Let's keep it compact.



##########
src/iceberg/update/set_snapshot.h:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+
+/// \file iceberg/update/set_snapshot.h
+/// \brief Sets the current snapshot directly or by rolling back.
+
+namespace iceberg {
+
+/// \brief Sets the current snapshot directly or by rolling back.
+class ICEBERG_EXPORT SetSnapshot : public PendingUpdate {
+ public:
+  static Result<std::shared_ptr<SetSnapshot>> Make(
+      std::shared_ptr<Transaction> transaction);
+
+  ~SetSnapshot() override;
+
+  /// \brief Sets the table's current state to a specific Snapshot identified 
by id.
+  ///
+  /// This method allows setting the current snapshot to <b>any</b> valid 
snapshot defined
+  /// in the table metadata, regardless of its relationship to the current 
state.
+  ///
+  /// \param snapshot_id The snapshot ID to set as current
+  /// \return Reference to this for method chaining
+  SetSnapshot& SetCurrentSnapshot(int64_t snapshot_id);
+
+  /// \brief Rolls back the table's state to the last Snapshot before the 
given timestamp.
+  ///
+  /// This method traverses the history of the current snapshot to find the 
most recent
+  /// ancestor that happened before the specified time.
+  ///
+  /// \param timestamp_ms A timestamp in milliseconds since the Unix epoch
+  /// \return Reference to this for method chaining

Review Comment:
   ```suggestion
     /// \brief Rolls back the table's state to the last Snapshot before the 
given timestamp.
   ```



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));
+  ICEBERG_BUILDER_CHECK(
+      is_ancestor,
+      "Cannot roll back to snapshot, not an ancestor of the current state: {}",
+      snapshot_id);
+
+  return SetCurrentSnapshot(snapshot_id);
+}
+
+Result<std::shared_ptr<Snapshot>> SetSnapshot::Apply() {
+  ICEBERG_RETURN_UNEXPECTED(CheckErrors());
+
+  const TableMetadata& base_metadata = transaction_->current();
+
+  // If no target snapshot was configured, return current state (NOOP)
+  if (!target_snapshot_id_.has_value()) {
+    return base_metadata.Snapshot();
+  }
+
+  // If this is a rollback, validate that the target is still an ancestor
+  if (is_rollback_) {
+    ICEBERG_ASSIGN_OR_RAISE(
+        bool is_ancestor,
+        SnapshotUtil::IsAncestorOf(*transaction_->table(), 
target_snapshot_id_.value()));
+    ICEBERG_CHECK(is_ancestor,
+                  "Cannot roll back to {}: not an ancestor of the current 
table state",
+                  target_snapshot_id_.value());
+  }
+
+  return base_metadata.SnapshotById(target_snapshot_id_.value());
+}
+
+Result<std::optional<std::shared_ptr<Snapshot>>> 
SetSnapshot::FindLatestAncestorOlderThan(
+    int64_t timestamp_ms) const {
+  ICEBERG_ASSIGN_OR_RAISE(auto ancestors,
+                          
SnapshotUtil::CurrentAncestors(*transaction_->table()));
+
+  int64_t snapshot_timestamp = 0;
+  std::shared_ptr<Snapshot> result = nullptr;
+
+  for (const auto& snapshot : ancestors) {

Review Comment:
   ```suggestion
     for (auto& snapshot : ancestors) {
   ```
   
   Do not use const so we can directly move this away like below.



##########
src/iceberg/update/set_snapshot.h:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+
+/// \file iceberg/update/set_snapshot.h
+/// \brief Sets the current snapshot directly or by rolling back.
+
+namespace iceberg {
+
+/// \brief Sets the current snapshot directly or by rolling back.
+class ICEBERG_EXPORT SetSnapshot : public PendingUpdate {
+ public:
+  static Result<std::shared_ptr<SetSnapshot>> Make(
+      std::shared_ptr<Transaction> transaction);
+
+  ~SetSnapshot() override;
+
+  /// \brief Sets the table's current state to a specific Snapshot identified 
by id.
+  ///
+  /// This method allows setting the current snapshot to <b>any</b> valid 
snapshot defined
+  /// in the table metadata, regardless of its relationship to the current 
state.
+  ///
+  /// \param snapshot_id The snapshot ID to set as current
+  /// \return Reference to this for method chaining
+  SetSnapshot& SetCurrentSnapshot(int64_t snapshot_id);
+
+  /// \brief Rolls back the table's state to the last Snapshot before the 
given timestamp.
+  ///
+  /// This method traverses the history of the current snapshot to find the 
most recent
+  /// ancestor that happened before the specified time.
+  ///
+  /// \param timestamp_ms A timestamp in milliseconds since the Unix epoch
+  /// \return Reference to this for method chaining
+  SetSnapshot& RollbackToTime(int64_t timestamp_ms);
+
+  /// \brief Rollback table's state to a specific Snapshot identified by id.
+  ///
+  /// This method strictly validates that the target snapshot is an ancestor 
of the
+  /// current table state.
+  ///
+  /// \param snapshot_id The snapshot ID to roll back to. Must be an ancestor 
of the
+  /// current snapshot
+  /// \return Reference to this for method chaining

Review Comment:
   ```suggestion
   ```



##########
src/iceberg/test/set_snapshot_test.cc:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <memory>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/test/matchers.h"
+#include "iceberg/test/update_test_base.h"
+#include "iceberg/transaction.h"
+
+namespace iceberg {
+
+// Test fixture for SetSnapshot tests
+class SetSnapshotTest : public UpdateTestBase {
+ protected:
+  // Snapshot IDs from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotId = 3051729675574597004;
+  static constexpr int64_t kCurrentSnapshotId = 3055729675574597004;
+
+  // Timestamps from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotTimestamp = 1515100955770;
+  static constexpr int64_t kCurrentSnapshotTimestamp = 1555100955770;
+};
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));

Review Comment:
   Let's create it directly from transaction.



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));
+  ICEBERG_BUILDER_CHECK(
+      is_ancestor,
+      "Cannot roll back to snapshot, not an ancestor of the current state: {}",
+      snapshot_id);
+
+  return SetCurrentSnapshot(snapshot_id);
+}
+
+Result<std::shared_ptr<Snapshot>> SetSnapshot::Apply() {
+  ICEBERG_RETURN_UNEXPECTED(CheckErrors());
+
+  const TableMetadata& base_metadata = transaction_->current();
+
+  // If no target snapshot was configured, return current state (NOOP)
+  if (!target_snapshot_id_.has_value()) {
+    return base_metadata.Snapshot();
+  }
+
+  // If this is a rollback, validate that the target is still an ancestor
+  if (is_rollback_) {
+    ICEBERG_ASSIGN_OR_RAISE(
+        bool is_ancestor,
+        SnapshotUtil::IsAncestorOf(*transaction_->table(), 
target_snapshot_id_.value()));

Review Comment:
   Same issue with `*transaction_->table()`



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));
+  ICEBERG_BUILDER_CHECK(
+      is_ancestor,
+      "Cannot roll back to snapshot, not an ancestor of the current state: {}",
+      snapshot_id);
+
+  return SetCurrentSnapshot(snapshot_id);
+}
+
+Result<std::shared_ptr<Snapshot>> SetSnapshot::Apply() {
+  ICEBERG_RETURN_UNEXPECTED(CheckErrors());
+
+  const TableMetadata& base_metadata = transaction_->current();
+
+  // If no target snapshot was configured, return current state (NOOP)
+  if (!target_snapshot_id_.has_value()) {
+    return base_metadata.Snapshot();
+  }
+
+  // If this is a rollback, validate that the target is still an ancestor
+  if (is_rollback_) {
+    ICEBERG_ASSIGN_OR_RAISE(
+        bool is_ancestor,
+        SnapshotUtil::IsAncestorOf(*transaction_->table(), 
target_snapshot_id_.value()));
+    ICEBERG_CHECK(is_ancestor,
+                  "Cannot roll back to {}: not an ancestor of the current 
table state",
+                  target_snapshot_id_.value());
+  }
+
+  return base_metadata.SnapshotById(target_snapshot_id_.value());
+}
+
+Result<std::optional<std::shared_ptr<Snapshot>>> 
SetSnapshot::FindLatestAncestorOlderThan(
+    int64_t timestamp_ms) const {
+  ICEBERG_ASSIGN_OR_RAISE(auto ancestors,
+                          
SnapshotUtil::CurrentAncestors(*transaction_->table()));

Review Comment:
   Same issue here.



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));
+  ICEBERG_BUILDER_CHECK(
+      is_ancestor,
+      "Cannot roll back to snapshot, not an ancestor of the current state: {}",
+      snapshot_id);
+
+  return SetCurrentSnapshot(snapshot_id);
+}
+
+Result<std::shared_ptr<Snapshot>> SetSnapshot::Apply() {
+  ICEBERG_RETURN_UNEXPECTED(CheckErrors());
+
+  const TableMetadata& base_metadata = transaction_->current();
+
+  // If no target snapshot was configured, return current state (NOOP)
+  if (!target_snapshot_id_.has_value()) {
+    return base_metadata.Snapshot();
+  }
+
+  // If this is a rollback, validate that the target is still an ancestor
+  if (is_rollback_) {
+    ICEBERG_ASSIGN_OR_RAISE(
+        bool is_ancestor,
+        SnapshotUtil::IsAncestorOf(*transaction_->table(), 
target_snapshot_id_.value()));
+    ICEBERG_CHECK(is_ancestor,
+                  "Cannot roll back to {}: not an ancestor of the current 
table state",
+                  target_snapshot_id_.value());
+  }
+
+  return base_metadata.SnapshotById(target_snapshot_id_.value());
+}
+
+Result<std::optional<std::shared_ptr<Snapshot>>> 
SetSnapshot::FindLatestAncestorOlderThan(
+    int64_t timestamp_ms) const {
+  ICEBERG_ASSIGN_OR_RAISE(auto ancestors,
+                          
SnapshotUtil::CurrentAncestors(*transaction_->table()));
+
+  int64_t snapshot_timestamp = 0;
+  std::shared_ptr<Snapshot> result = nullptr;
+
+  for (const auto& snapshot : ancestors) {
+    if (snapshot == nullptr) {
+      continue;
+    }
+
+    int64_t snap_timestamp_ms = UnixMsFromTimePointMs(snapshot->timestamp_ms);

Review Comment:
   Can we declare `snapshot_timestamp` as `TimePointMs` so we don't need to 
convert this for every snapshot?



##########
src/iceberg/test/set_snapshot_test.cc:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <memory>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/test/matchers.h"
+#include "iceberg/test/update_test_base.h"
+#include "iceberg/transaction.h"
+
+namespace iceberg {
+
+// Test fixture for SetSnapshot tests
+class SetSnapshotTest : public UpdateTestBase {
+ protected:
+  // Snapshot IDs from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotId = 3051729675574597004;
+  static constexpr int64_t kCurrentSnapshotId = 3055729675574597004;
+
+  // Timestamps from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotTimestamp = 1515100955770;
+  static constexpr int64_t kCurrentSnapshotTimestamp = 1555100955770;
+};
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the older snapshot
+  set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotToCurrentSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the current snapshot (no-op)
+  set_snapshot->SetCurrentSnapshot(kCurrentSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotInvalid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to set to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->SetCurrentSnapshot(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to the oldest snapshot (which is an ancestor)
+  set_snapshot->RollbackTo(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToInvalidSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->RollbackTo(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeValidOldestSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a time between the two snapshots
+  // This should select the oldest snapshot
+  int64_t time_between = (kOldestSnapshotTimestamp + 
kCurrentSnapshotTimestamp) / 2;
+  set_snapshot->RollbackToTime(time_between);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeBeforeAnySnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a time before any snapshot
+  int64_t time_before_all = kOldestSnapshotTimestamp - 1000000;
+  set_snapshot->RollbackToTime(time_before_all);
+
+  // Should fail - no snapshot older than the specified time
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("no valid snapshot older than"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeExactMatch) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a timestamp just after the oldest snapshot
+  // This should return the oldest snapshot (the latest one before this time)
+  int64_t time_just_after_oldest = kOldestSnapshotTimestamp + 1;
+  set_snapshot->RollbackToTime(time_just_after_oldest);
+
+  // Apply and verify - should return the oldest snapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, ApplyWithoutChanges) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Apply without making any changes (NOOP)
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+
+  // Should return current snapshot
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, MethodChaining) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Test that methods return reference for chaining
+  // Note: Only the last operation should take effect
+  auto& result1 = set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+  EXPECT_EQ(&result1, set_snapshot.get());
+}
+
+TEST_F(SetSnapshotTest, CommitSuccess) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set to oldest snapshot
+  set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+
+  // Commit the change
+  EXPECT_THAT(set_snapshot->Commit(), IsOk());
+
+  // Commit the transaction
+  ICEBERG_UNWRAP_OR_FAIL(auto updated_table, txn->Commit());
+
+  // Verify the current snapshot was changed
+  ICEBERG_UNWRAP_OR_FAIL(auto reloaded, catalog_->LoadTable(table_ident_));
+  ICEBERG_UNWRAP_OR_FAIL(auto current_snapshot, reloaded->current_snapshot());
+  EXPECT_EQ(current_snapshot->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, CommitEmptyUpdate) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Commit without making any changes (NOOP)
+  EXPECT_THAT(set_snapshot->Commit(), IsOk());
+
+  // Commit the transaction
+  ICEBERG_UNWRAP_OR_FAIL(auto updated_table, txn->Commit());
+
+  // Verify the current snapshot remained the same
+  ICEBERG_UNWRAP_OR_FAIL(auto reloaded, catalog_->LoadTable(table_ident_));
+  ICEBERG_UNWRAP_OR_FAIL(auto current_snapshot, reloaded->current_snapshot());
+  EXPECT_EQ(current_snapshot->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, KindReturnsSetSnapshot) {

Review Comment:
   This does not deserve a separate test. We can test it in any above test.



##########
src/iceberg/update/set_snapshot.cc:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <cstdint>
+#include <memory>
+#include <optional>
+
+#include "iceberg/snapshot.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/util/error_collector.h"
+#include "iceberg/util/macros.h"
+#include "iceberg/util/snapshot_util_internal.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SetSnapshot>> SetSnapshot::Make(
+    std::shared_ptr<Transaction> transaction) {
+  ICEBERG_PRECHECK(transaction != nullptr,
+                   "Cannot create SetSnapshot without a transaction");
+  return std::shared_ptr<SetSnapshot>(new SetSnapshot(std::move(transaction)));
+}
+
+SetSnapshot::SetSnapshot(std::shared_ptr<Transaction> transaction)
+    : PendingUpdate(std::move(transaction)) {}
+
+SetSnapshot::~SetSnapshot() = default;
+
+SetSnapshot& SetSnapshot::SetCurrentSnapshot(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  target_snapshot_id_ = snapshot_id;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackToTime(int64_t timestamp_ms) {
+  // Find the latest snapshot by timestamp older than timestamp_ms
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto snapshot_opt,
+                                   FindLatestAncestorOlderThan(timestamp_ms));
+
+  ICEBERG_BUILDER_CHECK(snapshot_opt.has_value(),
+                        "Cannot roll back, no valid snapshot older than: {}",
+                        timestamp_ms);
+
+  target_snapshot_id_ = snapshot_opt.value()->snapshot_id;
+  is_rollback_ = true;
+
+  return *this;
+}
+
+SetSnapshot& SetSnapshot::RollbackTo(int64_t snapshot_id) {
+  // Validate that the snapshot exists
+  auto snapshot_result = base().SnapshotById(snapshot_id);
+  ICEBERG_BUILDER_CHECK(snapshot_result.has_value(),
+                        "Cannot roll back to unknown snapshot id: {}", 
snapshot_id);
+
+  // Validate that the snapshot is an ancestor of the current state
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(
+      bool is_ancestor, SnapshotUtil::IsAncestorOf(*transaction_->table(), 
snapshot_id));
+  ICEBERG_BUILDER_CHECK(
+      is_ancestor,
+      "Cannot roll back to snapshot, not an ancestor of the current state: {}",
+      snapshot_id);
+
+  return SetCurrentSnapshot(snapshot_id);
+}
+
+Result<std::shared_ptr<Snapshot>> SetSnapshot::Apply() {
+  ICEBERG_RETURN_UNEXPECTED(CheckErrors());
+
+  const TableMetadata& base_metadata = transaction_->current();
+
+  // If no target snapshot was configured, return current state (NOOP)
+  if (!target_snapshot_id_.has_value()) {
+    return base_metadata.Snapshot();
+  }
+
+  // If this is a rollback, validate that the target is still an ancestor
+  if (is_rollback_) {
+    ICEBERG_ASSIGN_OR_RAISE(
+        bool is_ancestor,
+        SnapshotUtil::IsAncestorOf(*transaction_->table(), 
target_snapshot_id_.value()));
+    ICEBERG_CHECK(is_ancestor,
+                  "Cannot roll back to {}: not an ancestor of the current 
table state",
+                  target_snapshot_id_.value());
+  }
+
+  return base_metadata.SnapshotById(target_snapshot_id_.value());
+}
+
+Result<std::optional<std::shared_ptr<Snapshot>>> 
SetSnapshot::FindLatestAncestorOlderThan(
+    int64_t timestamp_ms) const {
+  ICEBERG_ASSIGN_OR_RAISE(auto ancestors,
+                          
SnapshotUtil::CurrentAncestors(*transaction_->table()));
+
+  int64_t snapshot_timestamp = 0;
+  std::shared_ptr<Snapshot> result = nullptr;
+
+  for (const auto& snapshot : ancestors) {
+    if (snapshot == nullptr) {
+      continue;
+    }
+
+    int64_t snap_timestamp_ms = UnixMsFromTimePointMs(snapshot->timestamp_ms);
+
+    if (snap_timestamp_ms < timestamp_ms && snap_timestamp_ms > 
snapshot_timestamp) {
+      result = snapshot;

Review Comment:
   ```suggestion
         result = std::move(snapshot);
   ```



##########
src/iceberg/test/set_snapshot_test.cc:
##########
@@ -0,0 +1,236 @@
+/*
+ * 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/update/set_snapshot.h"
+
+#include <memory>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/test/matchers.h"
+#include "iceberg/test/update_test_base.h"
+#include "iceberg/transaction.h"
+
+namespace iceberg {
+
+// Test fixture for SetSnapshot tests
+class SetSnapshotTest : public UpdateTestBase {
+ protected:
+  // Snapshot IDs from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotId = 3051729675574597004;
+  static constexpr int64_t kCurrentSnapshotId = 3055729675574597004;
+
+  // Timestamps from TableMetadataV2Valid.json
+  static constexpr int64_t kOldestSnapshotTimestamp = 1515100955770;
+  static constexpr int64_t kCurrentSnapshotTimestamp = 1555100955770;
+};
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the older snapshot
+  set_snapshot->SetCurrentSnapshot(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotToCurrentSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Set current snapshot to the current snapshot (no-op)
+  set_snapshot->SetCurrentSnapshot(kCurrentSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, SetCurrentSnapshotInvalid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to set to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->SetCurrentSnapshot(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToValid) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to the oldest snapshot (which is an ancestor)
+  set_snapshot->RollbackTo(kOldestSnapshotId);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToInvalidSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a non-existent snapshot
+  int64_t invalid_snapshot_id = 9999999999999999;
+  set_snapshot->RollbackTo(invalid_snapshot_id);
+
+  // Should fail during Apply
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("unknown snapshot id"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeValidOldestSnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a time between the two snapshots
+  // This should select the oldest snapshot
+  int64_t time_between = (kOldestSnapshotTimestamp + 
kCurrentSnapshotTimestamp) / 2;
+  set_snapshot->RollbackToTime(time_between);
+
+  // Apply and verify
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeBeforeAnySnapshot) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Try to rollback to a time before any snapshot
+  int64_t time_before_all = kOldestSnapshotTimestamp - 1000000;
+  set_snapshot->RollbackToTime(time_before_all);
+
+  // Should fail - no snapshot older than the specified time
+  auto result = set_snapshot->Apply();
+  EXPECT_THAT(result, IsError(ErrorKind::kValidationFailed));
+  EXPECT_THAT(result, HasErrorMessage("no valid snapshot older than"));
+}
+
+TEST_F(SetSnapshotTest, RollbackToTimeExactMatch) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Rollback to a timestamp just after the oldest snapshot
+  // This should return the oldest snapshot (the latest one before this time)
+  int64_t time_just_after_oldest = kOldestSnapshotTimestamp + 1;
+  set_snapshot->RollbackToTime(time_just_after_oldest);
+
+  // Apply and verify - should return the oldest snapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kOldestSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, ApplyWithoutChanges) {
+  // Create transaction and SetSnapshot
+  ICEBERG_UNWRAP_OR_FAIL(auto txn, table_->NewTransaction());
+  ICEBERG_UNWRAP_OR_FAIL(auto set_snapshot, SetSnapshot::Make(txn));
+
+  // Apply without making any changes (NOOP)
+  ICEBERG_UNWRAP_OR_FAIL(auto result, set_snapshot->Apply());
+
+  // Should return current snapshot
+  EXPECT_NE(result, nullptr);
+  EXPECT_EQ(result->snapshot_id, kCurrentSnapshotId);
+}
+
+TEST_F(SetSnapshotTest, MethodChaining) {

Review Comment:
   I don't think we need this trivial test



-- 
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