From 829eb752145a1121c9b3c146a67dbb7a1cf80179 Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Fri, 17 Oct 2025 16:39:47 +0800 Subject: [PATCH] test: add more table metadata interop test --- src/iceberg/table_metadata.h | 3 +- src/iceberg/test/metadata_serde_test.cc | 498 +++++++++++++++++------- src/iceberg/test/test_common.cc | 13 +- src/iceberg/test/test_common.h | 4 + 4 files changed, 368 insertions(+), 150 deletions(-) diff --git a/src/iceberg/table_metadata.h b/src/iceberg/table_metadata.h index 427447a10..f4f8d7296 100644 --- a/src/iceberg/table_metadata.h +++ b/src/iceberg/table_metadata.h @@ -135,7 +135,8 @@ struct ICEBERG_EXPORT TableMetadata { /// \brief Get the snapshot of this table with the given id Result> SnapshotById(int64_t snapshot_id) const; - friend bool operator==(const TableMetadata& lhs, const TableMetadata& rhs); + ICEBERG_EXPORT friend bool operator==(const TableMetadata& lhs, + const TableMetadata& rhs); }; /// \brief Returns a string representation of a SnapshotLogEntry diff --git a/src/iceberg/test/metadata_serde_test.cc b/src/iceberg/test/metadata_serde_test.cc index 73e5dd3a5..744a55c75 100644 --- a/src/iceberg/test/metadata_serde_test.cc +++ b/src/iceberg/test/metadata_serde_test.cc @@ -17,13 +17,10 @@ * under the License. */ -#include -#include #include #include #include -#include #include "iceberg/partition_field.h" #include "iceberg/partition_spec.h" @@ -32,174 +29,387 @@ #include "iceberg/snapshot.h" #include "iceberg/sort_field.h" #include "iceberg/sort_order.h" +#include "iceberg/statistics_file.h" #include "iceberg/table_metadata.h" #include "iceberg/transform.h" #include "iceberg/type.h" +#include "matchers.h" #include "test_common.h" namespace iceberg { namespace { -class MetadataSerdeTest : public ::testing::Test { - protected: - void SetUp() override {} -}; +void ReadTableMetadataExpectError(const std::string& file_name, + const std::string& expected_error_substr) { + auto result = ReadTableMetadata(file_name); + ASSERT_FALSE(result.has_value()) << "Expected parsing to fail for " << file_name; + EXPECT_THAT(result, HasErrorMessage(expected_error_substr)); +} + +void AssertSchema(const TableMetadata& metadata, const Schema& expected_schema) { + auto schema = metadata.Schema(); + ASSERT_TRUE(schema.has_value()); + EXPECT_EQ(*(schema.value().get()), expected_schema); +} + +void AssertSchemaById(const TableMetadata& metadata, int32_t schema_id, + const Schema& expected_schema) { + auto schema = metadata.SchemaById(schema_id); + ASSERT_TRUE(schema.has_value()); + EXPECT_EQ(*(schema.value().get()), expected_schema); +} + +void AssertPartitionSpec(const TableMetadata& metadata, + const PartitionSpec& expected_spec) { + auto partition_spec = metadata.PartitionSpec(); + ASSERT_TRUE(partition_spec.has_value()); + EXPECT_EQ(*(partition_spec.value().get()), expected_spec); +} + +void AssertSortOrder(const TableMetadata& metadata, + const SortOrder& expected_sort_order) { + auto sort_order = metadata.SortOrder(); + ASSERT_TRUE(sort_order.has_value()); + EXPECT_EQ(*(sort_order.value().get()), expected_sort_order); +} + +void AssertSnapshot(const TableMetadata& metadata, const Snapshot& expected_snapshot) { + auto snapshot = metadata.Snapshot(); + ASSERT_TRUE(snapshot.has_value()); + EXPECT_EQ(*snapshot.value(), expected_snapshot); +} + +void AssertSnapshotById(const TableMetadata& metadata, int64_t snapshot_id, + const Snapshot& expected_snapshot) { + auto snapshot = metadata.SnapshotById(snapshot_id); + ASSERT_TRUE(snapshot.has_value()); + EXPECT_EQ(*snapshot.value(), expected_snapshot); +} } // namespace -TEST_F(MetadataSerdeTest, DeserializeV1Valid) { +TEST(MetadataSerdeTest, DeserializeV1Valid) { std::unique_ptr metadata; ASSERT_NO_FATAL_FAILURE(ReadTableMetadata("TableMetadataV1Valid.json", &metadata)); - EXPECT_EQ(metadata->format_version, 1); - EXPECT_EQ(metadata->table_uuid, "d20125c8-7284-442c-9aea-15fee620737c"); - EXPECT_EQ(metadata->location, "s3://bucket/test/location"); - EXPECT_EQ(metadata->last_updated_ms.time_since_epoch().count(), 1602638573874); - EXPECT_EQ(metadata->last_column_id, 3); - EXPECT_EQ(metadata->current_snapshot_id, -1); - - // Compare schema - EXPECT_EQ(metadata->current_schema_id, std::nullopt); - std::vector schema_fields; - schema_fields.emplace_back(/*field_id=*/1, "x", iceberg::int64(), - /*optional=*/false); - schema_fields.emplace_back(/*field_id=*/2, "y", iceberg::int64(), - /*optional=*/false); - schema_fields.emplace_back(/*field_id=*/3, "z", iceberg::int64(), - /*optional=*/false); - auto expected_schema = - std::make_shared(schema_fields, /*schema_id=*/std::nullopt); - auto schema = metadata->Schema(); - ASSERT_TRUE(schema.has_value()); - EXPECT_EQ(*(schema.value().get()), *expected_schema); - - // Compare partition spec - std::vector partition_fields; - partition_fields.emplace_back(/*source_id=*/1, /*field_id=*/1000, /*name=*/"x", - Transform::Identity()); - auto expected_spec = - std::make_shared(expected_schema, /*spec_id=*/0, partition_fields); - auto partition_spec = metadata->PartitionSpec(); - ASSERT_TRUE(partition_spec.has_value()); - EXPECT_EQ(*(partition_spec.value().get()), *expected_spec); - auto snapshot = metadata->Snapshot(); - ASSERT_FALSE(snapshot.has_value()); + auto expected_schema = std::make_shared( + std::vector{SchemaField::MakeRequired(1, "x", int64()), + SchemaField::MakeRequired(2, "y", int64()), + SchemaField::MakeRequired(3, "z", int64())}, + /*schema_id=*/std::nullopt); + + auto expected_spec = std::make_shared( + expected_schema, /*spec_id=*/0, + std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", + Transform::Identity())}); + + TableMetadata expected{ + .format_version = 1, + .table_uuid = "d20125c8-7284-442c-9aea-15fee620737c", + .location = "s3://bucket/test/location", + .last_sequence_number = 0, + .last_updated_ms = TimePointMsFromUnixMs(1602638573874).value(), + .last_column_id = 3, + .schemas = {expected_schema}, + .current_schema_id = std::nullopt, + .partition_specs = {expected_spec}, + .default_spec_id = 0, + .last_partition_id = 1000, + .current_snapshot_id = -1, + .sort_orders = {SortOrder::Unsorted()}, + .default_sort_order_id = 0, + .next_row_id = 0, + }; + + ASSERT_EQ(*metadata, expected); + AssertSchema(*metadata, *expected_schema); + AssertPartitionSpec(*metadata, *expected_spec); + ASSERT_FALSE(metadata->Snapshot().has_value()); } -TEST_F(MetadataSerdeTest, DeserializeV2Valid) { +TEST(MetadataSerdeTest, DeserializeV2Valid) { std::unique_ptr metadata; ASSERT_NO_FATAL_FAILURE(ReadTableMetadata("TableMetadataV2Valid.json", &metadata)); - EXPECT_EQ(metadata->format_version, 2); - EXPECT_EQ(metadata->table_uuid, "9c12d441-03fe-4693-9a96-a0705ddf69c1"); - EXPECT_EQ(metadata->location, "s3://bucket/test/location"); - EXPECT_EQ(metadata->last_updated_ms.time_since_epoch().count(), 1602638573590); - EXPECT_EQ(metadata->last_column_id, 3); - - // Compare schema - EXPECT_EQ(metadata->current_schema_id, 1); - std::vector schema_fields; - schema_fields.emplace_back(/*field_id=*/1, "x", iceberg::int64(), - /*optional=*/false); - schema_fields.emplace_back(/*field_id=*/2, "y", iceberg::int64(), - /*optional=*/false); - schema_fields.emplace_back(/*field_id=*/3, "z", iceberg::int64(), - /*optional=*/false); - auto expected_schema = std::make_shared(schema_fields, /*schema_id=*/1); - auto schema = metadata->Schema(); - ASSERT_TRUE(schema.has_value()); - EXPECT_EQ(*(schema.value().get()), *expected_schema); - - // schema with ID 1 - auto schema_v1 = metadata->SchemaById(1); - ASSERT_TRUE(schema_v1.has_value()); - EXPECT_EQ(*(schema_v1.value().get()), *expected_schema); - - // schema with ID 0 - auto expected_schema_v0 = std::make_shared( - std::vector{schema_fields.at(0)}, /*schema_id=*/0); - auto schema_v0 = metadata->SchemaById(0); - ASSERT_TRUE(schema_v0.has_value()); - EXPECT_EQ(*(schema_v0.value().get()), *expected_schema_v0); - - // Compare partition spec - EXPECT_EQ(metadata->default_spec_id, 0); - std::vector partition_fields; - partition_fields.emplace_back(/*source_id=*/1, /*field_id=*/1000, /*name=*/"x", - Transform::Identity()); + auto expected_schema_1 = std::make_shared( + std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + /*optional=*/false)}, + /*schema_id=*/0); + + auto expected_schema_2 = std::make_shared( + std::vector{SchemaField::MakeRequired(1, "x", int64()), + SchemaField::MakeRequired(2, "y", int64()), + SchemaField::MakeRequired(3, "z", int64())}, + /*schema_id=*/1); + + auto expected_spec = std::make_shared( + expected_schema_2, /*spec_id=*/0, + std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", + Transform::Identity())}); + + auto expected_sort_order = std::make_shared( + /*order_id=*/3, + std::vector{SortField(/*source_id=*/2, Transform::Identity(), + SortDirection::kAscending, NullOrder::kFirst), + SortField(/*source_id=*/3, Transform::Bucket(4), + SortDirection::kDescending, NullOrder::kLast)}); + + auto expected_snapshot_1 = std::make_shared(Snapshot{ + .snapshot_id = 3051729675574597004, + .sequence_number = 0, + .timestamp_ms = TimePointMsFromUnixMs(1515100955770).value(), + .manifest_list = "s3://a/b/1.avro", + .summary = {{"operation", "append"}}, + }); + + auto expected_snapshot_2 = std::make_shared(Snapshot{ + .snapshot_id = 3055729675574597004, + .parent_snapshot_id = 3051729675574597004, + .sequence_number = 1, + .timestamp_ms = TimePointMsFromUnixMs(1555100955770).value(), + .manifest_list = "s3://a/b/2.avro", + .summary = {{"operation", "append"}}, + .schema_id = 1, + }); + + TableMetadata expected{ + .format_version = 2, + .table_uuid = "9c12d441-03fe-4693-9a96-a0705ddf69c1", + .location = "s3://bucket/test/location", + .last_sequence_number = 34, + .last_updated_ms = TimePointMsFromUnixMs(1602638573590).value(), + .last_column_id = 3, + .schemas = {expected_schema_1, expected_schema_2}, + .current_schema_id = 1, + .partition_specs = {expected_spec}, + .default_spec_id = 0, + .last_partition_id = 1000, + .current_snapshot_id = 3055729675574597004, + .snapshots = {expected_snapshot_1, expected_snapshot_2}, + .snapshot_log = {SnapshotLogEntry{ + .timestamp_ms = TimePointMsFromUnixMs(1515100955770).value(), + .snapshot_id = 3051729675574597004}, + SnapshotLogEntry{ + .timestamp_ms = TimePointMsFromUnixMs(1555100955770).value(), + .snapshot_id = 3055729675574597004}}, + .sort_orders = {expected_sort_order}, + .default_sort_order_id = 3, + .refs = {{"main", std::make_shared( + SnapshotRef{.snapshot_id = 3055729675574597004, + .retention = SnapshotRef::Branch{}})}}, + .next_row_id = 0, + }; + + ASSERT_EQ(*metadata, expected); + AssertSchema(*metadata, *expected_schema_2); + AssertSchemaById(*metadata, 0, *expected_schema_1); + AssertSchemaById(*metadata, 1, *expected_schema_2); + AssertPartitionSpec(*metadata, *expected_spec); + AssertSortOrder(*metadata, *expected_sort_order); + AssertSnapshot(*metadata, *expected_snapshot_2); + AssertSnapshotById(*metadata, 3051729675574597004, *expected_snapshot_1); + AssertSnapshotById(*metadata, 3055729675574597004, *expected_snapshot_2); +} + +TEST(MetadataSerdeTest, DeserializeV2ValidMinimal) { + std::unique_ptr metadata; + ASSERT_NO_FATAL_FAILURE( + ReadTableMetadata("TableMetadataV2ValidMinimal.json", &metadata)); + + auto expected_schema = std::make_shared( + std::vector{SchemaField::MakeRequired(1, "x", int64()), + SchemaField::MakeRequired(2, "y", int64(), "comment"), + SchemaField::MakeRequired(3, "z", int64())}, + /*schema_id=*/0); + + auto expected_spec = std::make_shared( + expected_schema, /*spec_id=*/0, + std::vector{PartitionField(/*source_id=*/1, /*field_id=*/1000, "x", + Transform::Identity())}); + + auto expected_sort_order = std::make_shared( + /*order_id=*/3, std::vector{ + SortField(/*source_id=*/2, Transform::Identity(), + SortDirection::kAscending, NullOrder::kFirst), + SortField(/*source_id=*/3, Transform::Bucket(4), + SortDirection::kDescending, NullOrder::kLast), + }); + + TableMetadata expected{ + .format_version = 2, + .table_uuid = "9c12d441-03fe-4693-9a96-a0705ddf69c1", + .location = "s3://bucket/test/location", + .last_sequence_number = 34, + .last_updated_ms = TimePointMsFromUnixMs(1602638573590).value(), + .last_column_id = 3, + .schemas = {expected_schema}, + .current_schema_id = 0, + .partition_specs = {expected_spec}, + .default_spec_id = 0, + .last_partition_id = 1000, + .current_snapshot_id = -1, + .sort_orders = {expected_sort_order}, + .default_sort_order_id = 3, + .next_row_id = 0, + }; + + ASSERT_EQ(*metadata, expected); + AssertSchema(*metadata, *expected_schema); + AssertPartitionSpec(*metadata, *expected_spec); + AssertSortOrder(*metadata, *expected_sort_order); + ASSERT_FALSE(metadata->Snapshot().has_value()); +} + +TEST(MetadataSerdeTest, DeserializeStatisticsFiles) { + std::unique_ptr metadata; + ASSERT_NO_FATAL_FAILURE( + ReadTableMetadata("TableMetadataStatisticsFiles.json", &metadata)); + + auto expected_schema = std::make_shared( + std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + /*optional=*/false)}, + /*schema_id=*/0); + auto expected_spec = std::make_shared(expected_schema, /*spec_id=*/0, - std::move(partition_fields)); - auto partition_spec = metadata->PartitionSpec(); - ASSERT_TRUE(partition_spec.has_value()); - EXPECT_EQ(*(partition_spec.value().get()), *expected_spec); - - // Compare sort order - EXPECT_EQ(metadata->default_sort_order_id, 3); - std::vector sort_fields; - sort_fields.emplace_back(/*source_id=*/2, Transform::Identity(), - SortDirection::kAscending, NullOrder::kFirst); - sort_fields.emplace_back(/*source_id=*/3, Transform::Bucket(4), - SortDirection::kDescending, NullOrder::kLast); - auto expected_sort_order = - std::make_shared(/*order_id=*/3, std::move(sort_fields)); - auto sort_order = metadata->SortOrder(); - ASSERT_TRUE(sort_order.has_value()); - EXPECT_EQ(*(sort_order.value().get()), *expected_sort_order); + std::vector{}); - // Compare snapshot - EXPECT_EQ(metadata->current_snapshot_id, 3055729675574597004); - auto snapshot = metadata->Snapshot(); - ASSERT_TRUE(snapshot.has_value()); - EXPECT_EQ(snapshot.value()->snapshot_id, 3055729675574597004); - - // Compare snapshots - std::vector expected_snapshots{ - { - .snapshot_id = 3051729675574597004, - .sequence_number = 0, - .timestamp_ms = TimePointMsFromUnixMs(1515100955770).value(), - .manifest_list = "s3://a/b/1.avro", - .summary = {{"operation", "append"}}, - }, - { + auto expected_snapshot = std::make_shared(Snapshot{ + .snapshot_id = 3055729675574597004, + .sequence_number = 1, + .timestamp_ms = TimePointMsFromUnixMs(1555100955770).value(), + .manifest_list = "s3://a/b/2.avro", + .summary = {{"operation", "append"}}, + .schema_id = 0, + }); + + auto expected_stats_file = std::make_shared(StatisticsFile{ + .snapshot_id = 3055729675574597004, + .path = "s3://a/b/stats.puffin", + .file_size_in_bytes = 413, + .file_footer_size_in_bytes = 42, + .blob_metadata = + { + BlobMetadata{ + .type = "ndv", + .source_snapshot_id = 3055729675574597004, + .source_snapshot_sequence_number = 1, + .fields = {1}, + .properties = {}, + }, + }, + }); + + TableMetadata expected{ + .format_version = 2, + .table_uuid = "9c12d441-03fe-4693-9a96-a0705ddf69c1", + .location = "s3://bucket/test/location", + .last_sequence_number = 34, + .last_updated_ms = TimePointMsFromUnixMs(1602638573590).value(), + .last_column_id = 3, + .schemas = {expected_schema}, + .current_schema_id = 0, + .partition_specs = {expected_spec}, + .default_spec_id = 0, + .last_partition_id = 1000, + .properties = {}, + .current_snapshot_id = 3055729675574597004, + .snapshots = {expected_snapshot}, + .snapshot_log = {}, + .metadata_log = {}, + .sort_orders = {SortOrder::Unsorted()}, + .default_sort_order_id = 0, + .refs = {{"main", std::make_shared( + SnapshotRef{.snapshot_id = 3055729675574597004, + .retention = SnapshotRef::Branch{}})}}, + .statistics = {expected_stats_file}, + .partition_statistics = {}, + .next_row_id = 0, + }; + + ASSERT_EQ(*metadata, expected); +} + +TEST(MetadataSerdeTest, DeserializePartitionStatisticsFiles) { + std::unique_ptr metadata; + ASSERT_NO_FATAL_FAILURE( + ReadTableMetadata("TableMetadataPartitionStatisticsFiles.json", &metadata)); + + TableMetadata expected{ + .format_version = 2, + .table_uuid = "9c12d441-03fe-4693-9a96-a0705ddf69c1", + .location = "s3://bucket/test/location", + .last_sequence_number = 34, + .last_updated_ms = TimePointMsFromUnixMs(1602638573590).value(), + .last_column_id = 3, + .schemas = {std::make_shared( + std::vector{SchemaField(/*field_id=*/1, "x", iceberg::int64(), + /*optional=*/false)}, + /*schema_id=*/0)}, + .current_schema_id = 0, + .partition_specs = {PartitionSpec::Unpartitioned()}, + .default_spec_id = 0, + .last_partition_id = 1000, + .properties = {}, + .current_snapshot_id = 3055729675574597004, + .snapshots = {std::make_shared(Snapshot{ .snapshot_id = 3055729675574597004, - .parent_snapshot_id = 3051729675574597004, .sequence_number = 1, .timestamp_ms = TimePointMsFromUnixMs(1555100955770).value(), .manifest_list = "s3://a/b/2.avro", .summary = {{"operation", "append"}}, - .schema_id = 1, - }}; - EXPECT_EQ(metadata->snapshots.size(), expected_snapshots.size()); - for (size_t i = 0; i < expected_snapshots.size(); ++i) { - EXPECT_EQ(*metadata->snapshots[i], expected_snapshots[i]); - } - - // snapshot with ID 3051729675574597004 - auto snapshot_v0 = metadata->SnapshotById(3051729675574597004); - ASSERT_TRUE(snapshot_v0.has_value()); - EXPECT_EQ(*snapshot_v0.value(), expected_snapshots[0]); - - // snapshot with ID 3055729675574597004 - auto snapshot_v1 = metadata->SnapshotById(3055729675574597004); - ASSERT_TRUE(snapshot_v1.has_value()); - EXPECT_EQ(*snapshot_v1.value(), expected_snapshots[1]); - - // Compare snapshot logs - std::vector expected_snapshot_log{ - { - .timestamp_ms = TimePointMsFromUnixMs(1515100955770).value(), - .snapshot_id = 3051729675574597004, - }, - { - .timestamp_ms = TimePointMsFromUnixMs(1555100955770).value(), - .snapshot_id = 3055729675574597004, - }}; - EXPECT_EQ(metadata->snapshot_log.size(), 2); - for (size_t i = 0; i < expected_snapshots.size(); ++i) { - EXPECT_EQ(metadata->snapshot_log[i], expected_snapshot_log[i]); - } + .schema_id = 0, + })}, + .snapshot_log = {}, + .metadata_log = {}, + .sort_orders = {SortOrder::Unsorted()}, + .default_sort_order_id = 0, + .refs = {{"main", std::make_shared( + SnapshotRef{.snapshot_id = 3055729675574597004, + .retention = SnapshotRef::Branch{}})}}, + .statistics = {}, + .partition_statistics = {std::make_shared( + PartitionStatisticsFile{.snapshot_id = 3055729675574597004, + .path = "s3://a/b/partition-stats.parquet", + .file_size_in_bytes = 43})}, + .next_row_id = 0, + }; + + ASSERT_EQ(*metadata, expected); +} + +TEST(MetadataSerdeTest, DeserializeUnsupportedVersion) { + ReadTableMetadataExpectError("TableMetadataUnsupportedVersion.json", + "Cannot read unsupported version"); +} + +TEST(MetadataSerdeTest, DeserializeV1MissingSchemaType) { + ReadTableMetadataExpectError("TableMetadataV1MissingSchemaType.json", "Missing 'type'"); +} + +TEST(MetadataSerdeTest, DeserializeV2CurrentSchemaNotFound) { + ReadTableMetadataExpectError("TableMetadataV2CurrentSchemaNotFound.json", + "Cannot find schema with current-schema-id"); +} + +TEST(MetadataSerdeTest, DeserializeV2MissingLastPartitionId) { + ReadTableMetadataExpectError("TableMetadataV2MissingLastPartitionId.json", + "last-partition-id must exist"); +} + +TEST(MetadataSerdeTest, DeserializeV2MissingPartitionSpecs) { + ReadTableMetadataExpectError("TableMetadataV2MissingPartitionSpecs.json", + "partition-specs must exist"); +} + +TEST(MetadataSerdeTest, DeserializeV2MissingSchemas) { + ReadTableMetadataExpectError("TableMetadataV2MissingSchemas.json", + "schemas must exist"); +} + +TEST(MetadataSerdeTest, DeserializeV2MissingSortOrder) { + ReadTableMetadataExpectError("TableMetadataV2MissingSortOrder.json", + "sort-orders must exist"); } } // namespace iceberg diff --git a/src/iceberg/test/test_common.cc b/src/iceberg/test/test_common.cc index 25fa8f2c7..8c992c9bd 100644 --- a/src/iceberg/test/test_common.cc +++ b/src/iceberg/test/test_common.cc @@ -21,7 +21,6 @@ #include #include -#include #include #include @@ -48,14 +47,18 @@ void ReadJsonFile(const std::string& file_name, std::string* content) { void ReadTableMetadata(const std::string& file_name, std::unique_ptr* metadata) { + auto result = ReadTableMetadata(file_name); + ASSERT_TRUE(result.has_value()) << "Failed to parse table metadata from " << file_name + << ": " << result.error().message; + *metadata = std::move(result.value()); +} + +Result> ReadTableMetadata(const std::string& file_name) { std::string json_content; ReadJsonFile(file_name, &json_content); nlohmann::json json = nlohmann::json::parse(json_content); - auto result = TableMetadataFromJson(json); - ASSERT_TRUE(result.has_value()) << "Failed to parse table metadata from " << file_name - << ": " << result.error().message; - *metadata = std::move(result.value()); + return TableMetadataFromJson(json); } } // namespace iceberg diff --git a/src/iceberg/test/test_common.h b/src/iceberg/test/test_common.h index a9dba8cad..e41a3f3e4 100644 --- a/src/iceberg/test/test_common.h +++ b/src/iceberg/test/test_common.h @@ -22,6 +22,7 @@ #include #include +#include "iceberg/result.h" #include "iceberg/type_fwd.h" namespace iceberg { @@ -36,4 +37,7 @@ void ReadJsonFile(const std::string& file_name, std::string* content); void ReadTableMetadata(const std::string& file_name, std::unique_ptr* metadata); +/// \brief Read table metadata from a JSON file and return the Result directly +Result> ReadTableMetadata(const std::string& file_name); + } // namespace iceberg