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


##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,
+/// data sequence number, file sequence number), these values should be 
inherited
+/// from the manifest file. This interface provides a way to apply such 
inheritance rules.
+class ICEBERG_EXPORT InheritableMetadata {
+ public:
+  virtual ~InheritableMetadata() = default;
+
+  /// \brief Apply inheritable metadata to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry with inherited metadata applied.

Review Comment:
   This is inconsistent



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,
+/// data sequence number, file sequence number), these values should be 
inherited
+/// from the manifest file. This interface provides a way to apply such 
inheritance rules.
+class ICEBERG_EXPORT InheritableMetadata {
+ public:
+  virtual ~InheritableMetadata() = default;
+
+  /// \brief Apply inheritable metadata to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry with inherited metadata applied.
+  virtual Status Apply(ManifestEntry& entry) = 0;
+};
+
+/// \brief Base implementation of InheritableMetadata that handles standard 
inheritance
+/// rules.
+class ICEBERG_EXPORT BaseInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Constructor for base inheritable metadata.
+  /// \param spec_id Partition spec ID from the manifest.
+  /// \param snapshot_id Snapshot ID from the manifest.
+  /// \param sequence_number Sequence number from the manifest.
+  /// \param manifest_location Path to the manifest file.
+  BaseInheritableMetadata(int32_t spec_id, int64_t snapshot_id, int64_t 
sequence_number,
+                          std::string manifest_location);
+
+  /// \brief Apply inheritance rules to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry.
+  Status Apply(ManifestEntry& entry) override;
+
+ private:
+  int32_t spec_id_;
+  int64_t snapshot_id_;
+  int64_t sequence_number_;
+  std::string manifest_location_;
+};
+
+/// \brief Empty implementation that applies no inheritance.
+class ICEBERG_EXPORT EmptyInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Apply no inheritance - returns the entry unchanged.
+  /// \param entry The manifest entry (unchanged).
+  /// \return The manifest entry.
+  Status Apply(ManifestEntry& entry) override;
+};
+
+/// \brief Metadata inheritance for copying manifests before commit.
+class ICEBERG_EXPORT CopyInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Constructor for copy metadata.
+  /// \param snapshot_id The snapshot ID to use for copying.
+  explicit CopyInheritableMetadata(int64_t snapshot_id);
+
+  /// \brief Apply copy inheritance rules.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry.

Review Comment:
   ditto



##########
src/iceberg/inheritable_metadata.cc:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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/inheritable_metadata.h"
+
+#include <cassert>
+#include <utility>
+
+#include "iceberg/manifest_entry.h"
+
+namespace iceberg {
+
+BaseInheritableMetadata::BaseInheritableMetadata(int32_t spec_id, int64_t 
snapshot_id,
+                                                 int64_t sequence_number,
+                                                 std::string manifest_location)
+    : spec_id_(spec_id),
+      snapshot_id_(snapshot_id),
+      sequence_number_(sequence_number),
+      manifest_location_(std::move(manifest_location)) {}
+
+Status BaseInheritableMetadata::Apply(ManifestEntry& entry) {
+  if (!entry.snapshot_id.has_value()) {
+    entry.snapshot_id = snapshot_id_;
+  }
+
+  // In v1 tables, the data sequence number is not persisted and can be safely 
defaulted
+  // to 0.
+  // In v2 tables, the data sequence number should be inherited iff the entry 
status
+  // is ADDED
+  if (!entry.sequence_number.has_value() &&
+      (sequence_number_ == 0 || entry.status == ManifestStatus::kAdded)) {
+    entry.sequence_number = sequence_number_;
+  }
+
+  // In v1 tables, the file sequence number is not persisted and can be safely 
defaulted
+  // to 0.
+  //  In v2 tables, the file sequence number should be inherited iff the entry 
status
+  // is ADDED
+  if (!entry.file_sequence_number.has_value() &&
+      (sequence_number_ == 0 || entry.status == ManifestStatus::kAdded)) {
+    entry.file_sequence_number = sequence_number_;
+  }
+
+  if (entry.data_file) {
+    entry.data_file->partition_spec_id = spec_id_;
+  }
+
+  return {};
+}
+
+Status EmptyInheritableMetadata::Apply(ManifestEntry& entry) {
+  if (!entry.snapshot_id.has_value()) {
+    return InvalidArgument(
+        "Entries must have explicit snapshot ids if inherited metadata is 
empty");
+  }
+  return {};
+}
+
+CopyInheritableMetadata::CopyInheritableMetadata(int64_t snapshot_id)
+    : snapshot_id_(snapshot_id) {}
+
+Status CopyInheritableMetadata::Apply(ManifestEntry& entry) {
+  entry.snapshot_id = snapshot_id_;
+  return {};
+}
+
+Result<std::unique_ptr<InheritableMetadata>> 
InheritableMetadataFactory::Empty() {
+  return std::make_unique<EmptyInheritableMetadata>();
+}
+
+Result<std::unique_ptr<InheritableMetadata>> 
InheritableMetadataFactory::FromManifest(
+    const ManifestFile& manifest) {
+  // Validate that the manifest has a snapshot ID assigned
+  if (manifest.added_snapshot_id == Snapshot::kInvalidSnapshotId) {
+    return InvalidArgument("Manifest file {} has no snapshot ID", 
manifest.manifest_path);

Review Comment:
   ```suggestion
       return InvalidManifest("Manifest file {} has no snapshot ID", 
manifest.manifest_path);
   ```



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,

Review Comment:
   ```suggestion
   /// When manifest entries have null values for certain fields (snapshot id,
   ```



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,
+/// data sequence number, file sequence number), these values should be 
inherited
+/// from the manifest file. This interface provides a way to apply such 
inheritance rules.
+class ICEBERG_EXPORT InheritableMetadata {
+ public:
+  virtual ~InheritableMetadata() = default;
+
+  /// \brief Apply inheritable metadata to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry with inherited metadata applied.
+  virtual Status Apply(ManifestEntry& entry) = 0;
+};
+
+/// \brief Base implementation of InheritableMetadata that handles standard 
inheritance
+/// rules.
+class ICEBERG_EXPORT BaseInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Constructor for base inheritable metadata.
+  /// \param spec_id Partition spec ID from the manifest.
+  /// \param snapshot_id Snapshot ID from the manifest.
+  /// \param sequence_number Sequence number from the manifest.
+  /// \param manifest_location Path to the manifest file.
+  BaseInheritableMetadata(int32_t spec_id, int64_t snapshot_id, int64_t 
sequence_number,
+                          std::string manifest_location);
+
+  /// \brief Apply inheritance rules to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry.
+  Status Apply(ManifestEntry& entry) override;
+
+ private:
+  int32_t spec_id_;
+  int64_t snapshot_id_;
+  int64_t sequence_number_;
+  std::string manifest_location_;
+};
+
+/// \brief Empty implementation that applies no inheritance.
+class ICEBERG_EXPORT EmptyInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Apply no inheritance - returns the entry unchanged.
+  /// \param entry The manifest entry (unchanged).
+  /// \return The manifest entry.

Review Comment:
   ditto



##########
test/manifest_reader_test.cc:
##########
@@ -46,7 +47,44 @@ class ManifestReaderV1Test : public TempFileTestBase {
     avro::RegisterLogicalTypes();
   }
 
-  std::vector<ManifestEntry> PrepareV1ManifestEntries() {
+  void TestManifestReading(const std::string& resource_name,
+                           const std::vector<ManifestEntry>& expected_entries,
+                           std::shared_ptr<Schema> partition_schema = nullptr) 
{
+    std::string path = GetResourcePath(resource_name);
+    auto manifest_reader_result = ManifestReader::Make(path, file_io_, 
partition_schema);
+    ASSERT_EQ(manifest_reader_result.has_value(), true)

Review Comment:
   ```suggestion
       ASSERT_TRUE(manifest_reader_result.has_value())
   ```



##########
src/iceberg/manifest_reader.cc:
##########
@@ -27,7 +29,27 @@
 
 namespace iceberg {
 
-Result<std::unique_ptr<ManifestReader>> ManifestReader::MakeReader(
+Result<std::unique_ptr<ManifestReader>> ManifestReader::Make(
+    const ManifestFile& manifest, std::shared_ptr<FileIO> file_io,
+    std::shared_ptr<Schema> partition_schema) {
+  auto manifest_entry_schema = 
ManifestEntry::TypeFromPartitionType(partition_schema);
+  std::shared_ptr<Schema> schema =
+      FromStructType(std::move(*manifest_entry_schema), std::nullopt);
+
+  ICEBERG_ASSIGN_OR_RAISE(
+      auto reader,
+      ReaderFactoryRegistry::Open(FileFormatType::kAvro, {.path = 
manifest.manifest_path,

Review Comment:
   Can we also set manifest.manifest_length to it?



##########
src/iceberg/manifest_reader_internal.cc:
##########
@@ -543,6 +541,13 @@ Result<std::vector<ManifestEntry>> 
ManifestReaderImpl::Entries() const {
       break;
     }
   }
+
+  // Apply inheritance to all entries
+  for (auto& entry : manifest_entries) {
+    auto status = inheritable_metadata_->Apply(entry);
+    ICEBERG_RETURN_UNEXPECTED(status);

Review Comment:
   ```suggestion
       ICEBERG_RETURN_UNEXPECTED(inheritable_metadata_->Apply(entry));
   ```



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,
+/// data sequence number, file sequence number), these values should be 
inherited
+/// from the manifest file. This interface provides a way to apply such 
inheritance rules.
+class ICEBERG_EXPORT InheritableMetadata {
+ public:
+  virtual ~InheritableMetadata() = default;
+
+  /// \brief Apply inheritable metadata to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry with inherited metadata applied.
+  virtual Status Apply(ManifestEntry& entry) = 0;
+};
+
+/// \brief Base implementation of InheritableMetadata that handles standard 
inheritance
+/// rules.
+class ICEBERG_EXPORT BaseInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Constructor for base inheritable metadata.
+  /// \param spec_id Partition spec ID from the manifest.
+  /// \param snapshot_id Snapshot ID from the manifest.
+  /// \param sequence_number Sequence number from the manifest.
+  /// \param manifest_location Path to the manifest file.
+  BaseInheritableMetadata(int32_t spec_id, int64_t snapshot_id, int64_t 
sequence_number,
+                          std::string manifest_location);
+
+  /// \brief Apply inheritance rules to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry.

Review Comment:
   ditto



##########
src/iceberg/inheritable_metadata.cc:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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/inheritable_metadata.h"
+
+#include <cassert>
+#include <utility>
+
+#include "iceberg/manifest_entry.h"
+
+namespace iceberg {
+
+BaseInheritableMetadata::BaseInheritableMetadata(int32_t spec_id, int64_t 
snapshot_id,
+                                                 int64_t sequence_number,
+                                                 std::string manifest_location)
+    : spec_id_(spec_id),
+      snapshot_id_(snapshot_id),
+      sequence_number_(sequence_number),
+      manifest_location_(std::move(manifest_location)) {}
+
+Status BaseInheritableMetadata::Apply(ManifestEntry& entry) {
+  if (!entry.snapshot_id.has_value()) {
+    entry.snapshot_id = snapshot_id_;
+  }
+
+  // In v1 tables, the data sequence number is not persisted and can be safely 
defaulted
+  // to 0.
+  // In v2 tables, the data sequence number should be inherited iff the entry 
status
+  // is ADDED
+  if (!entry.sequence_number.has_value() &&
+      (sequence_number_ == 0 || entry.status == ManifestStatus::kAdded)) {
+    entry.sequence_number = sequence_number_;
+  }
+
+  // In v1 tables, the file sequence number is not persisted and can be safely 
defaulted
+  // to 0.
+  //  In v2 tables, the file sequence number should be inherited iff the entry 
status
+  // is ADDED
+  if (!entry.file_sequence_number.has_value() &&
+      (sequence_number_ == 0 || entry.status == ManifestStatus::kAdded)) {
+    entry.file_sequence_number = sequence_number_;
+  }
+
+  if (entry.data_file) {
+    entry.data_file->partition_spec_id = spec_id_;
+  }
+
+  return {};
+}
+
+Status EmptyInheritableMetadata::Apply(ManifestEntry& entry) {
+  if (!entry.snapshot_id.has_value()) {
+    return InvalidArgument(

Review Comment:
   ```suggestion
       return InvalidManifest(
   ```



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>

Review Comment:
   ```suggestion
   #include "iceberg/result.h"
   ```



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"

Review Comment:
   ```suggestion
   #include "iceberg/type_fwd.h"
   ```
   
   Can we use forward declaration?



##########
test/manifest_reader_test.cc:
##########
@@ -154,51 +173,64 @@ class ManifestReaderV2Test : public TempFileTestBase {
          {3, {'d', 'a', 't', 'a', '_', 'c', 'o', 'n', 't', 'e', 'n', 't', '_', 
'4'}},
          {4, {0x14, 0xae, 0x47, 0xe1, 0x7a, 0x8c, 0x7c, 0x40}}}};
 
+    DataFile data_file{.file_path = test_dir_prefix + paths[0],
+                       .file_format = FileFormatType::kParquet,
+                       .record_count = record_counts[0],
+                       .file_size_in_bytes = file_sizes[0],
+                       .column_sizes = {{1, 56}, {2, 73}, {3, 66}, {4, 67}},
+                       .value_counts = {{1, 4}, {2, 4}, {3, 4}, {4, 4}},
+                       .null_value_counts = {{1, 0}, {2, 0}, {3, 0}, {4, 0}},
+                       .nan_value_counts = {{4, 0}},
+                       .lower_bounds = lower_bounds[0],
+                       .upper_bounds = upper_bounds[0],
+                       .key_metadata = {},
+                       .split_offsets = {4},
+                       .equality_ids = {},
+                       .sort_order_id = 0,
+                       .first_row_id = std::nullopt,
+                       .referenced_data_file = std::nullopt,
+                       .content_offset = std::nullopt,
+                       .content_size_in_bytes = std::nullopt};
+
+    if (partition_spec_id.has_value()) {
+      data_file.partition_spec_id = partition_spec_id.value();
+    }
+
     manifest_entries.emplace_back(
         ManifestEntry{.status = ManifestStatus::kAdded,
                       .snapshot_id = 679879563479918846LL,
-                      .sequence_number = std::nullopt,
-                      .file_sequence_number = std::nullopt,
-                      .data_file = std::make_shared<DataFile>(
-                          DataFile{.file_path = test_dir_prefix + paths[0],
-                                   .file_format = FileFormatType::kParquet,
-                                   .record_count = record_counts[0],
-                                   .file_size_in_bytes = file_sizes[0],
-                                   .column_sizes = {{1, 56}, {2, 73}, {3, 66}, 
{4, 67}},
-                                   .value_counts = {{1, 4}, {2, 4}, {3, 4}, 
{4, 4}},
-                                   .null_value_counts = {{1, 0}, {2, 0}, {3, 
0}, {4, 0}},
-                                   .nan_value_counts = {{4, 0}},
-                                   .lower_bounds = lower_bounds[0],
-                                   .upper_bounds = upper_bounds[0],
-                                   .key_metadata = {},
-                                   .split_offsets = {4},
-                                   .equality_ids = {},
-                                   .sort_order_id = 0,
-                                   .first_row_id = std::nullopt,
-                                   .referenced_data_file = std::nullopt,
-                                   .content_offset = std::nullopt,
-                                   .content_size_in_bytes = std::nullopt})});
+                      .sequence_number = sequence_number,
+                      .file_sequence_number = sequence_number,
+                      .data_file = std::make_shared<DataFile>(data_file)});
     return manifest_entries;
   }
 
-  std::shared_ptr<::arrow::fs::LocalFileSystem> local_fs_;
-  std::shared_ptr<FileIO> file_io_;
-};
-
-TEST_F(ManifestReaderV2Test, V2NonPartitionedBasicTest) {
-  std::string path = 
GetResourcePath("2ddf1bc9-830b-4015-aced-c060df36f150-m0.avro");
+  std::vector<ManifestEntry> PrepareNonPartitionedTestData() {
+    return CreateV2TestData();
+  }
 
-  auto manifest_reader_result = ManifestReader::MakeReader(path, file_io_, 
nullptr);
-  ASSERT_EQ(manifest_reader_result.has_value(), true)
-      << manifest_reader_result.error().message;
+  std::vector<ManifestEntry> PrepareMetadataInheritanceTestData() {
+    return CreateV2TestData(15, 12);

Review Comment:
   ```suggestion
       return CreateV2TestData(/*sequence_number=*/15, /*partition_spec_id*/12);
   ```
   
   For better readability.



##########
src/iceberg/inheritable_metadata.h:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+/// \file iceberg/inheritable_metadata.h
+/// Metadata inheritance system for manifest entries.
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include <iceberg/result.h>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/manifest_entry.h"
+#include "iceberg/manifest_list.h"
+
+namespace iceberg {
+
+/// \brief Interface for applying inheritable metadata to manifest entries.
+///
+/// When manifest entries have null values for certain fields (snapshot_id,
+/// data sequence number, file sequence number), these values should be 
inherited
+/// from the manifest file. This interface provides a way to apply such 
inheritance rules.
+class ICEBERG_EXPORT InheritableMetadata {
+ public:
+  virtual ~InheritableMetadata() = default;
+
+  /// \brief Apply inheritable metadata to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry with inherited metadata applied.
+  virtual Status Apply(ManifestEntry& entry) = 0;
+};
+
+/// \brief Base implementation of InheritableMetadata that handles standard 
inheritance
+/// rules.
+class ICEBERG_EXPORT BaseInheritableMetadata : public InheritableMetadata {
+ public:
+  /// \brief Constructor for base inheritable metadata.
+  /// \param spec_id Partition spec ID from the manifest.
+  /// \param snapshot_id Snapshot ID from the manifest.
+  /// \param sequence_number Sequence number from the manifest.
+  /// \param manifest_location Path to the manifest file.
+  BaseInheritableMetadata(int32_t spec_id, int64_t snapshot_id, int64_t 
sequence_number,
+                          std::string manifest_location);
+
+  /// \brief Apply inheritance rules to a manifest entry.
+  /// \param entry The manifest entry to modify.
+  /// \return The modified manifest entry.

Review Comment:
   BTW, override functions do not need docstirngs IMO



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to