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


##########
src/iceberg/update/snapshot_manager.h:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 <memory>
+#include <string>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+/// \brief API for managing snapshots and snapshot references.
+///
+/// Allows rolling table data back to a state at an older snapshot, 
cherry-picking
+/// snapshots, and managing branches and tags.
+class ICEBERG_EXPORT SnapshotManager : public PendingUpdate {
+ public:
+  /// \brief Create a SnapshotManager for a table.
+  ///
+  /// \param table_name The name of the table
+  /// \param table The table to manage snapshots for
+  /// \return A new SnapshotManager instance, or an error if the table doesn't 
exist
+  static Result<std::shared_ptr<SnapshotManager>> Make(const std::string& 
table_name,

Review Comment:
   Can we remove this ctor? `transaction` internally holds a `table` which has 
`table_identifier_` to get its name? We are slightly different than Java 
because we always create a transaction instance in the `Table` instance to 
create any update.



##########
src/iceberg/update/snapshot_manager.h:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 <memory>
+#include <string>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+/// \brief API for managing snapshots and snapshot references.
+///
+/// Allows rolling table data back to a state at an older snapshot, 
cherry-picking
+/// snapshots, and managing branches and tags.
+class ICEBERG_EXPORT SnapshotManager : public PendingUpdate {
+ public:
+  /// \brief Create a SnapshotManager for a table.
+  ///
+  /// \param table_name The name of the table
+  /// \param table The table to manage snapshots for
+  /// \return A new SnapshotManager instance, or an error if the table doesn't 
exist
+  static Result<std::shared_ptr<SnapshotManager>> Make(const std::string& 
table_name,
+                                                       std::shared_ptr<Table> 
table);
+
+  /// \brief Create a SnapshotManager from an existing transaction.
+  ///
+  /// \param transaction The transaction to use
+  /// \return A new SnapshotManager instance
+  static Result<std::shared_ptr<SnapshotManager>> Make(
+      std::shared_ptr<Transaction> transaction);
+
+  ~SnapshotManager() override;
+
+  // TODO(xxx): is this correct?
+  Kind kind() const final { return Kind::kUpdateSnapshotReference; }

Review Comment:
   We need to pay attention to 
https://github.com/apache/iceberg/blob/f23486ffd7499d2b2f0bfd8ce7552c361fd86796/core/src/main/java/org/apache/iceberg/BaseTransaction.java#L150-L153.
 `SnapshotManager` has its own commit logic so it looks tricky here.



##########
src/iceberg/type_fwd.h:
##########
@@ -184,6 +184,7 @@ class TableProperties;
 /// \brief Table update.
 class TableMetadataBuilder;
 class TableUpdate;
+class SnapshotManager;

Review Comment:
   Should we put it at line 195 below?



##########
src/iceberg/update/snapshot_update.h:
##########
@@ -76,6 +76,18 @@ class ICEBERG_EXPORT SnapshotUpdate : public PendingUpdate {
     return self;
   }
 
+  /// \brief Perform operations on a particular branch
+  ///
+  /// \param branch Which is name of SnapshotRef of type branch
+  /// \return Reference to this for method chaining
+  auto& ToBranch(this auto& self, const std::string& branch) {

Review Comment:
   Should we rename `SetTargetBranch` to `ToBranch` as they are equivalent?



##########
src/iceberg/update/snapshot_manager.cc:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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/snapshot_manager.h"
+
+#include <memory>
+#include <string>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/table.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/update/fast_append.h"
+#include "iceberg/update/set_snapshot.h"
+#include "iceberg/update/update_snapshot_reference.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SnapshotManager>> SnapshotManager::Make(
+    const std::string& table_name, std::shared_ptr<Table> table) {
+  if (table == nullptr) {
+    return InvalidArgument("Table cannot be null");
+  }
+  if (table->metadata() == nullptr) {
+    return InvalidArgument("Cannot manage snapshots: table {} does not exist",
+                           table_name);
+  }
+  // Create a transaction first
+  ICEBERG_ASSIGN_OR_RAISE(auto transaction,
+                          Transaction::Make(table, Transaction::Kind::kUpdate,
+                                            /*auto_commit=*/false));
+  auto manager = std::shared_ptr<SnapshotManager>(
+      new SnapshotManager(std::move(transaction), /*is_external=*/false));
+  return manager;
+}
+
+Result<std::shared_ptr<SnapshotManager>> SnapshotManager::Make(
+    std::shared_ptr<Transaction> transaction) {
+  if (transaction == nullptr) {
+    return InvalidArgument("Invalid input transaction: null");
+  }
+  return std::shared_ptr<SnapshotManager>(
+      new SnapshotManager(std::move(transaction), /*is_external=*/true));
+}
+
+SnapshotManager::SnapshotManager(std::shared_ptr<Transaction> transaction,
+                                 bool is_external)
+    : PendingUpdate(transaction), is_external_transaction_(is_external) {}
+
+SnapshotManager::~SnapshotManager() = default;
+
+SnapshotManager& SnapshotManager::Cherrypick(int64_t snapshot_id) {
+  ICEBERG_BUILDER_RETURN_IF_ERROR(CommitIfRefUpdatesExist());
+  // TODO(anyone): Implement cherrypick operation
+  ICEBERG_BUILDER_CHECK(false, "Cherrypick operation not yet implemented");
+  return *this;
+}
+
+SnapshotManager& SnapshotManager::SetCurrentSnapshot(int64_t snapshot_id) {
+  ICEBERG_BUILDER_RETURN_IF_ERROR(CommitIfRefUpdatesExist());
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto set_snapshot, 
transaction_->NewSetSnapshot());
+  set_snapshot->SetCurrentSnapshot(snapshot_id);
+  ICEBERG_BUILDER_RETURN_IF_ERROR(set_snapshot->Commit());
+  return *this;
+}
+
+SnapshotManager& SnapshotManager::RollbackToTime(TimePointMs timestamp_ms) {
+  ICEBERG_BUILDER_RETURN_IF_ERROR(CommitIfRefUpdatesExist());
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto set_snapshot, 
transaction_->NewSetSnapshot());
+  set_snapshot->RollbackToTime(UnixMsFromTimePointMs(timestamp_ms));
+  ICEBERG_BUILDER_RETURN_IF_ERROR(set_snapshot->Commit());
+  return *this;
+}
+
+SnapshotManager& SnapshotManager::RollbackTo(int64_t snapshot_id) {
+  ICEBERG_BUILDER_RETURN_IF_ERROR(CommitIfRefUpdatesExist());
+  ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto set_snapshot, 
transaction_->NewSetSnapshot());
+  set_snapshot->RollbackTo(snapshot_id);
+  ICEBERG_BUILDER_RETURN_IF_ERROR(set_snapshot->Commit());
+  return *this;
+}
+
+SnapshotManager& SnapshotManager::CreateBranch(const std::string& name) {
+  if (base().current_snapshot_id != kInvalidSnapshotId) {
+    ICEBERG_BUILDER_ASSIGN_OR_RETURN(auto current_snapshot, base().Snapshot());
+    if (current_snapshot != nullptr) {

Review Comment:
   It should be an error if current_snapshot_id exists but snapshot is null.



##########
src/iceberg/update/snapshot_manager.h:
##########
@@ -0,0 +1,211 @@
+/*
+ * 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 <memory>
+#include <string>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/type_fwd.h"
+#include "iceberg/update/pending_update.h"
+#include "iceberg/util/timepoint.h"
+
+namespace iceberg {
+
+/// \brief API for managing snapshots and snapshot references.
+///
+/// Allows rolling table data back to a state at an older snapshot, 
cherry-picking
+/// snapshots, and managing branches and tags.
+class ICEBERG_EXPORT SnapshotManager : public PendingUpdate {
+ public:
+  /// \brief Create a SnapshotManager for a table.
+  ///
+  /// \param table_name The name of the table
+  /// \param table The table to manage snapshots for
+  /// \return A new SnapshotManager instance, or an error if the table doesn't 
exist
+  static Result<std::shared_ptr<SnapshotManager>> Make(const std::string& 
table_name,
+                                                       std::shared_ptr<Table> 
table);
+
+  /// \brief Create a SnapshotManager from an existing transaction.
+  ///
+  /// \param transaction The transaction to use
+  /// \return A new SnapshotManager instance
+  static Result<std::shared_ptr<SnapshotManager>> Make(
+      std::shared_ptr<Transaction> transaction);
+
+  ~SnapshotManager() override;
+
+  // TODO(xxx): is this correct?
+  Kind kind() const final { return Kind::kUpdateSnapshotReference; }
+
+  /// \brief Apply supported changes in given snapshot and create a new 
snapshot which
+  /// will be set as the current snapshot on commit.
+  ///
+  /// \param snapshot_id A snapshot ID whose changes to apply
+  /// \return Reference to this for method chaining
+  SnapshotManager& Cherrypick(int64_t snapshot_id);
+
+  /// \brief Roll this table's data back to a specific Snapshot identified by 
id.
+  ///
+  /// \param snapshot_id Long id of the snapshot to roll back table data to
+  /// \return Reference to this for method chaining
+  SnapshotManager& SetCurrentSnapshot(int64_t snapshot_id);
+
+  /// \brief Roll this table's data back to the last Snapshot before the given 
timestamp.
+  ///
+  /// \param timestamp_ms A timestamp in milliseconds
+  /// \return Reference to this for method chaining
+  SnapshotManager& RollbackToTime(TimePointMs timestamp_ms);
+
+  /// \brief Rollback table's state to a specific Snapshot identified by id.
+  ///
+  /// \param snapshot_id Long id of snapshot id to roll back table to. Must be 
an ancestor
+  /// of the current snapshot
+  /// \return Reference to this for method chaining
+  SnapshotManager& RollbackTo(int64_t snapshot_id);
+
+  /// \brief Create a new branch. The branch will point to current snapshot if 
the current
+  /// snapshot is not NULL. Otherwise, the branch will point to a newly 
created empty
+  /// snapshot.
+  ///
+  /// \param name Branch name
+  /// \return Reference to this for method chaining
+  SnapshotManager& CreateBranch(const std::string& name);
+
+  /// \brief Create a new branch pointing to the given snapshot id.
+  ///
+  /// \param name Branch name
+  /// \param snapshot_id ID of the snapshot which will be the head of the 
branch
+  /// \return Reference to this for method chaining
+  SnapshotManager& CreateBranch(const std::string& name, int64_t snapshot_id);
+
+  /// \brief Create a new tag pointing to the given snapshot id.
+  ///
+  /// \param name Tag name
+  /// \param snapshot_id Snapshot ID for the head of the new tag
+  /// \return Reference to this for method chaining
+  SnapshotManager& CreateTag(const std::string& name, int64_t snapshot_id);
+
+  /// \brief Remove a branch by name.
+  ///
+  /// \param name Branch name
+  /// \return Reference to this for method chaining
+  SnapshotManager& RemoveBranch(const std::string& name);
+
+  /// \brief Remove the tag with the given name.
+  ///
+  /// \param name Tag name
+  /// \return Reference to this for method chaining
+  SnapshotManager& RemoveTag(const std::string& name);
+
+  /// \brief Replaces the tag with the given name to point to the specified 
snapshot.
+  ///
+  /// \param name Tag to replace
+  /// \param snapshot_id New snapshot id for the given tag
+  /// \return Reference to this for method chaining
+  SnapshotManager& ReplaceTag(const std::string& name, int64_t snapshot_id);
+
+  /// \brief Replaces the branch with the given name to point to the specified 
snapshot.
+  ///
+  /// \param name Branch to replace
+  /// \param snapshot_id New snapshot id for the given branch
+  /// \return Reference to this for method chaining
+  SnapshotManager& ReplaceBranch(const std::string& name, int64_t snapshot_id);
+
+  /// \brief Replaces the from branch to point to the to snapshot. The to will 
remain
+  /// unchanged, and from branch will retain its retention properties. If the 
from branch
+  /// does not exist, it will be created with default retention properties.
+  ///
+  /// \param from Branch to replace
+  /// \param to The branch from should be replaced with
+  /// \return Reference to this for method chaining
+  SnapshotManager& ReplaceBranch(const std::string& from, const std::string& 
to);
+
+  /// \brief Performs a fast-forward of from up to the to snapshot if from is 
an ancestor
+  /// of to. The to will remain unchanged, and from will retain its retention 
properties.
+  /// If the from branch does not exist, it will be created with default 
retention
+  /// properties.
+  ///
+  /// \param from Branch to fast-forward
+  /// \param to Ref for the from branch to be fast forwarded to
+  /// \return Reference to this for method chaining
+  SnapshotManager& FastForwardBranch(const std::string& from, const 
std::string& to);
+
+  /// \brief Rename a branch.
+  ///
+  /// \param name Name of branch to rename
+  /// \param new_name The desired new name of the branch
+  /// \return Reference to this for method chaining
+  SnapshotManager& RenameBranch(const std::string& name, const std::string& 
new_name);
+
+  /// \brief Updates the minimum number of snapshots to keep for a branch.
+  ///
+  /// \param branch_name Branch name
+  /// \param min_snapshots_to_keep Minimum number of snapshots to retain on 
the branch
+  /// \return Reference to this for method chaining
+  SnapshotManager& SetMinSnapshotsToKeep(const std::string& branch_name,
+                                         int32_t min_snapshots_to_keep);
+
+  /// \brief Updates the max snapshot age for a branch.
+  ///
+  /// \param branch_name Branch name
+  /// \param max_snapshot_age_ms Maximum snapshot age in milliseconds to 
retain on branch
+  /// \return Reference to this for method chaining
+  SnapshotManager& SetMaxSnapshotAgeMs(const std::string& branch_name,
+                                       int64_t max_snapshot_age_ms);
+
+  /// \brief Updates the retention policy for a reference.
+  ///
+  /// \param name Reference name
+  /// \param max_ref_age_ms Retention age in milliseconds of the tag reference 
itself
+  /// \return Reference to this for method chaining
+  SnapshotManager& SetMaxRefAgeMs(const std::string& name, int64_t 
max_ref_age_ms);
+
+  /// \brief Apply the pending changes and return the current snapshot.
+  ///
+  /// \return The current snapshot after applying changes, or an error
+  Result<std::shared_ptr<Snapshot>> Apply();
+
+  /// \brief Commit all pending changes.
+  ///
+  /// \return Status indicating success or failure
+  Status Commit() override;
+
+ private:
+  /// \brief Constructor for creating a SnapshotManager with a transaction.
+  ///
+  /// \param transaction The transaction to use
+  /// \param is_external Whether this is an external transaction (true) or 
created
+  /// internally (false)
+  SnapshotManager(std::shared_ptr<Transaction> transaction, bool is_external);
+
+  /// \brief Get or create the UpdateSnapshotReference operation.
+  Result<std::shared_ptr<UpdateSnapshotReference>> 
UpdateSnapshotReferencesOperation();
+
+  /// \brief Commit any pending reference updates if they exist.
+  Status CommitIfRefUpdatesExist();
+
+  bool is_external_transaction_;

Review Comment:
   I don't think we need this. `Transaction::auto_commit_` has already taken 
good care of it.



##########
src/iceberg/update/snapshot_manager.cc:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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/snapshot_manager.h"
+
+#include <memory>
+#include <string>
+
+#include "iceberg/result.h"
+#include "iceberg/snapshot.h"
+#include "iceberg/table.h"
+#include "iceberg/table_metadata.h"
+#include "iceberg/transaction.h"
+#include "iceberg/update/fast_append.h"
+#include "iceberg/update/set_snapshot.h"
+#include "iceberg/update/update_snapshot_reference.h"
+#include "iceberg/util/macros.h"
+
+namespace iceberg {
+
+Result<std::shared_ptr<SnapshotManager>> SnapshotManager::Make(
+    const std::string& table_name, std::shared_ptr<Table> table) {
+  if (table == nullptr) {
+    return InvalidArgument("Table cannot be null");
+  }
+  if (table->metadata() == nullptr) {
+    return InvalidArgument("Cannot manage snapshots: table {} does not exist",
+                           table_name);
+  }
+  // Create a transaction first
+  ICEBERG_ASSIGN_OR_RAISE(auto transaction,
+                          Transaction::Make(table, Transaction::Kind::kUpdate,
+                                            /*auto_commit=*/false));
+  auto manager = std::shared_ptr<SnapshotManager>(
+      new SnapshotManager(std::move(transaction), /*is_external=*/false));
+  return manager;
+}
+
+Result<std::shared_ptr<SnapshotManager>> SnapshotManager::Make(
+    std::shared_ptr<Transaction> transaction) {
+  if (transaction == nullptr) {
+    return InvalidArgument("Invalid input transaction: null");
+  }
+  return std::shared_ptr<SnapshotManager>(
+      new SnapshotManager(std::move(transaction), /*is_external=*/true));
+}
+
+SnapshotManager::SnapshotManager(std::shared_ptr<Transaction> transaction,
+                                 bool is_external)
+    : PendingUpdate(transaction), is_external_transaction_(is_external) {}
+
+SnapshotManager::~SnapshotManager() = default;
+
+SnapshotManager& SnapshotManager::Cherrypick(int64_t snapshot_id) {
+  ICEBERG_BUILDER_RETURN_IF_ERROR(CommitIfRefUpdatesExist());
+  // TODO(anyone): Implement cherrypick operation

Review Comment:
   I guess the reason is the lack of `CherryPickOperation` implementation.



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