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


##########
src/iceberg/partition_summary_internal.h:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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 "iceberg/expression/literal.h"
+#include "iceberg/result.h"
+#include "iceberg/type_fwd.h"
+
+namespace iceberg {
+
+class PartitionFieldStats {
+ public:
+  explicit PartitionFieldStats(const std::shared_ptr<Type>& type) : 
type_(type) {}
+
+  Status Update(const Literal& value);
+
+  PartitionFieldSummary Finish() const;
+
+ private:
+  std::shared_ptr<Type> type_{nullptr};
+  bool contains_null_{false};
+  bool contains_nan_{false};
+  std::optional<Literal> lower_bound_;
+  std::optional<Literal> upper_bound_;
+};
+
+class PartitionSummary {
+ public:
+  /// \brief Update the partition summary with partition.
+  Status Update(const std::vector<Literal>& partition);

Review Comment:
   ```suggestion
     /// \brief Update the partition summary with partition values.
     Status Update(const std::vector<Literal>& partition_values);
   ```
   
   `partition` was a little bit vague in its meaning.



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {

Review Comment:
   I think we need to support the allowed schema evolution rule here. Otherwise 
we end up with losing partition stats if field change like `int`->`long` or 
`float`->`double` happens.



##########
src/iceberg/manifest_writer.h:
##########
@@ -103,6 +124,10 @@ class ICEBERG_EXPORT ManifestWriter {
   static constexpr int64_t kBatchSize = 1024;
   std::unique_ptr<Writer> writer_;
   std::unique_ptr<ManifestEntryAdapter> adapter_;
+  bool closed_{false};
+  std::string manifest_location_;
+  // TODO(zhjwpku): consider passing key metadata when makeing ManifestWriter.
+  std::vector<uint8_t> key_metadata_;

Review Comment:
   We don't need this in the near future.



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();
+  }
+  if (upper_bound_) {
+    summary.upper_bound = upper_bound_->Serialize().value();
+  }
+  return summary;
+}
+
+Status PartitionSummary::Update(const std::vector<Literal>& partition) {
+  if (partition.size() != field_stats_.size()) {
+    return InvalidArgument("partition size does not match field stats size");
+  }
+
+  for (size_t i = 0; i < partition.size(); i++) {
+    ICEBERG_RETURN_UNEXPECTED(field_stats_[i].Update(partition[i]));

Review Comment:
   Perhaps we can call `partition[i].CastTo()` to handle the schema evolution 
here so `PartitionFieldStats::Update` can enforce strong type equality? This 
looks simpler.



##########
src/iceberg/manifest_adapter.h:
##########
@@ -61,18 +60,45 @@ class ICEBERG_EXPORT ManifestAdapter {
 /// Implemented by different versions with version-specific schemas.
 class ICEBERG_EXPORT ManifestEntryAdapter : public ManifestAdapter {
  public:
-  ManifestEntryAdapter(std::shared_ptr<PartitionSpec> partition_spec,
+  ManifestEntryAdapter(std::optional<int64_t> snapshot_id_,
+                       std::shared_ptr<PartitionSpec> partition_spec,
                        std::shared_ptr<Schema> current_schema, ManifestContent 
content);
 
   ~ManifestEntryAdapter() override;
 
-  virtual Status Append(const ManifestEntry& entry) = 0;

Review Comment:
   The main purpose of the adapter is:
   1. Convert this `ManifestEntry` and then append to arrow array. 
   2. Deal with differences among table format versions.
   
   I'd suggest keeping the signature unchanged. Perhaps all changes in this 
file should be moved to manifest_writer.h/cc.



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();
+  }
+  if (upper_bound_) {
+    summary.upper_bound = upper_bound_->Serialize().value();
+  }
+  return summary;
+}
+
+Status PartitionSummary::Update(const std::vector<Literal>& partition) {
+  if (partition.size() != field_stats_.size()) {
+    return InvalidArgument("partition size does not match field stats size");
+  }
+
+  for (size_t i = 0; i < partition.size(); i++) {
+    ICEBERG_RETURN_UNEXPECTED(field_stats_[i].Update(partition[i]));
+  }
+  return {};
+}
+
+std::vector<PartitionFieldSummary> PartitionSummary::Summaries() const {
+  std::vector<PartitionFieldSummary> summaries;
+  for (const auto& field_stat : field_stats_) {
+    summaries.push_back(field_stat.Finish());
+  }
+  return summaries;
+}
+
+Result<std::unique_ptr<PartitionSummary>> PartitionSummary::Make(

Review Comment:
   If there is no validation in the `Make` function, we don't need to add it. 
It is more convenient to directly use the constructor.
   
   This is not a strong opinion. Perhaps we can validate the partition type in 
the future.



##########
src/iceberg/manifest_entry.h:
##########
@@ -315,6 +306,17 @@ struct ICEBERG_EXPORT ManifestEntry {
   inline static const SchemaField kFileSequenceNumber =
       SchemaField::MakeOptional(4, "file_sequence_number", iceberg::int64());
 
+  /// \brief Check if this manifest entry is deleted.
+  constexpr bool IsAlive() const {
+    return status == ManifestStatus::kAdded || status == 
ManifestStatus::kExisting;
+  }
+
+  /// \brief Create a copy of this manifest entry.
+  ManifestEntry Copy() const {

Review Comment:
   Do we actually need this function? Can we just use `operator=`?



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"

Review Comment:
   Why adding `_internal` suffix to it? Isn't it a public api? 



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();

Review Comment:
   We need to handle any possible error returned by `lower_bound_->Serialize()` 
and `upper_bound_->Serialize()` below. Perhaps changing the function to return 
`Result< PartitionFieldSummary>` as well?



##########
src/iceberg/manifest_writer.h:
##########
@@ -54,9 +69,15 @@ class ICEBERG_EXPORT ManifestWriter {
   /// \brief Close writer and flush to storage.
   Status Close();
 
+  /// \brief Get the metrics of written manifest file.

Review Comment:
   ```suggestion
     /// \brief Get the metrics of written manifest file.
     /// \note Only valid after the file is closed.
   ```



##########
src/iceberg/manifest_reader_internal.cc:
##########
@@ -367,13 +367,16 @@ Status ParseDataFile(const std::shared_ptr<StructType>& 
data_file_schema,
           return InvalidManifest("Field:{} should be a struct.", field_name);
         }
         if (view_of_file_field->n_children > 0) {
-          auto view_of_partition = view_of_file_field->children[0];
-          for (int64_t row_idx = 0; row_idx < view_of_partition->length; 
row_idx++) {
-            if (ArrowArrayViewIsNull(view_of_partition, row_idx)) {
-              break;
+          for (int64_t partition_idx = 0; partition_idx < 
view_of_file_field->n_children;
+               partition_idx++) {
+            auto view_of_partition = 
view_of_file_field->children[partition_idx];

Review Comment:
   This is the fix to a bug that multiple partition fields are not correctly 
handled?



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();
+  }
+  if (upper_bound_) {
+    summary.upper_bound = upper_bound_->Serialize().value();
+  }
+  return summary;
+}
+
+Status PartitionSummary::Update(const std::vector<Literal>& partition) {
+  if (partition.size() != field_stats_.size()) {
+    return InvalidArgument("partition size does not match field stats size");

Review Comment:
   Let's also include the value in the error message.



##########
src/iceberg/manifest_list.h:
##########
@@ -198,6 +198,12 @@ struct ICEBERG_EXPORT ManifestFile {
   bool operator==(const ManifestFile& other) const = default;
 
   static const std::shared_ptr<Schema>& Type();
+
+  /// \brief Create a copy of this manifest file.
+  ManifestFile Copy() const {

Review Comment:
   Same question for `operator=`



##########
src/iceberg/partition_summary_internal.cc:
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/partition_summary_internal.h"
+
+#include <memory>
+
+#include "iceberg/expression/literal.h"
+#include "iceberg/manifest_list.h"
+#include "iceberg/result.h"
+
+namespace iceberg {
+
+Status PartitionFieldStats::Update(const Literal& value) {
+  if (type_->type_id() != value.type()->type_id()) {
+    return InvalidArgument("value is not compatible with type");
+  }
+
+  if (value.IsNull()) {
+    contains_null_ = true;
+    return {};
+  }
+
+  if (value.IsNan()) {
+    contains_nan_ = true;
+    return {};
+  }
+
+  if (!lower_bound_ || value < *lower_bound_) {
+    lower_bound_ = value;
+  }
+  if (!upper_bound_ || value > *upper_bound_) {
+    upper_bound_ = value;
+  }
+  return {};
+}
+
+PartitionFieldSummary PartitionFieldStats::Finish() const {
+  PartitionFieldSummary summary;
+  summary.contains_null = contains_null_;
+  summary.contains_nan = contains_nan_;
+  if (lower_bound_) {
+    summary.lower_bound = lower_bound_->Serialize().value();
+  }
+  if (upper_bound_) {
+    summary.upper_bound = upper_bound_->Serialize().value();
+  }
+  return summary;
+}
+
+Status PartitionSummary::Update(const std::vector<Literal>& partition) {
+  if (partition.size() != field_stats_.size()) {
+    return InvalidArgument("partition size does not match field stats size");
+  }
+
+  for (size_t i = 0; i < partition.size(); i++) {
+    ICEBERG_RETURN_UNEXPECTED(field_stats_[i].Update(partition[i]));
+  }
+  return {};
+}
+
+std::vector<PartitionFieldSummary> PartitionSummary::Summaries() const {
+  std::vector<PartitionFieldSummary> summaries;

Review Comment:
   nit: call reserve on `summaries`.



##########
src/iceberg/manifest_writer.h:
##########
@@ -54,9 +69,15 @@ class ICEBERG_EXPORT ManifestWriter {
   /// \brief Close writer and flush to storage.
   Status Close();
 
+  /// \brief Get the metrics of written manifest file.
+  std::optional<Metrics> metrics() const;
+
   /// \brief Get the content of the manifest.
   ManifestContent content() const;
 
+  /// \brief Get the ManifestFile object.
+  Result<ManifestFile> ToManifestFile() const;

Review Comment:
   I think we should be consistent for `metrics()` and `ToManifestFile()`. What 
about using the following:
   
   ```cpp
   Result<Metrics> metrics() const;
   Result<ManifestFile> ToManifestFile() const;  // unchanged
   ```
   
   We just return invalid state error when file is not closed.



##########
src/iceberg/metrics.h:
##########
@@ -32,12 +32,12 @@ namespace iceberg {
 /// \brief Iceberg file format metrics
 struct ICEBERG_EXPORT Metrics {
   int64_t row_count = 0;

Review Comment:
   ```suggestion
     std::optional<int64_t> row_count = 0;
   ```
   
   Should we make it optional?



##########
src/iceberg/manifest_writer.cc:
##########
@@ -171,7 +211,8 @@ Status ManifestListWriter::Add(const ManifestFile& file) {
     ICEBERG_RETURN_UNEXPECTED(writer_->Write(array));
     ICEBERG_RETURN_UNEXPECTED(adapter_->StartAppending());
   }
-  return adapter_->Append(file);
+  auto copy = file.Copy();
+  return adapter_->Append(copy);

Review Comment:
   ```suggestion
     return adapter_->Append(file);
   ```
   
   Please revert change for `ManifestListWriter`.



##########
src/iceberg/manifest_writer.h:
##########
@@ -54,9 +69,15 @@ class ICEBERG_EXPORT ManifestWriter {
   /// \brief Close writer and flush to storage.
   Status Close();
 
+  /// \brief Get the metrics of written manifest file.
+  std::optional<Metrics> metrics() const;
+
   /// \brief Get the content of the manifest.
   ManifestContent content() const;
 
+  /// \brief Get the ManifestFile object.
+  Result<ManifestFile> ToManifestFile() const;

Review Comment:
   Perhaps we should also change `FileWriter::metrics` to return 
`Result<Metrics>` as well and return not supported error for some writers.



##########
src/iceberg/manifest_writer.h:
##########
@@ -36,15 +38,28 @@ namespace iceberg {
 class ICEBERG_EXPORT ManifestWriter {
  public:
   ManifestWriter(std::unique_ptr<Writer> writer,
-                 std::unique_ptr<ManifestEntryAdapter> adapter)
-      : writer_(std::move(writer)), adapter_(std::move(adapter)) {}
+                 std::unique_ptr<ManifestEntryAdapter> adapter,
+                 std::string_view manifest_location)
+      : writer_(std::move(writer)),
+        adapter_(std::move(adapter)),
+        manifest_location_(manifest_location) {}
 
   ~ManifestWriter() = default;
 
-  /// \brief Write manifest entry to file.
+  /// \brief Add a new Manifest entry.
   /// \param entry Manifest entry to write.
   /// \return Status::OK() if entry was written successfully
-  Status Add(const ManifestEntry& entry);
+  Status AddEntry(const ManifestEntry& entry);

Review Comment:
   What about adding following functions to deal with different use cases:
   
   ```cpp
     // Write the entry that all its fields are populated correctly.
     Status WriteEntry(const ManifestEntry& entry);
   
     // Write the input as a new entry to add.
     Status WriteAddedEntry(std::shared_ptr<DataFile> file,
                            std::optional<int64_t> data_sequence_number);
     Status WriteAddedEntry(const ManifestEntry& entry);
   
     // Write the input as an existing entry.
     Status WriteExistingEntry(std::shared_ptr<DataFile> file, int64_t 
file_snapshot_id,
                               int64_t data_sequence_number,
                               std::optional<int64_t> file_sequence_number);
     Status WriteExistingEntry(const ManifestEntry& entry);
   
     // Write the input as a deleted entry.
     Status WriteDeletedEntry(std::shared_ptr<DataFile> file, int64_t 
data_sequence_number,
                              std::optional<int64_t> file_sequence_number);
     Status WriteDeletedEntry(const ManifestEntry& entry);
   
   ```



##########
src/iceberg/manifest_writer.h:
##########
@@ -36,15 +38,28 @@ namespace iceberg {
 class ICEBERG_EXPORT ManifestWriter {
  public:
   ManifestWriter(std::unique_ptr<Writer> writer,
-                 std::unique_ptr<ManifestEntryAdapter> adapter)
-      : writer_(std::move(writer)), adapter_(std::move(adapter)) {}
+                 std::unique_ptr<ManifestEntryAdapter> adapter,
+                 std::string_view manifest_location)
+      : writer_(std::move(writer)),
+        adapter_(std::move(adapter)),
+        manifest_location_(manifest_location) {}
 
   ~ManifestWriter() = default;
 
-  /// \brief Write manifest entry to file.
+  /// \brief Add a new Manifest entry.
   /// \param entry Manifest entry to write.
   /// \return Status::OK() if entry was written successfully
-  Status Add(const ManifestEntry& entry);
+  Status AddEntry(const ManifestEntry& entry);

Review Comment:
   `WriteEntry` can be the delegate function to accept writes from other 
functions. Inside each `WriteXXXEntry` function, we can just create a new entry 
and the copy should be cheap.



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