From 2066f993e937b0a04de9d68a9ff16ff2bd12fee9 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Sat, 3 Jan 2026 15:00:48 +0000 Subject: [PATCH 1/2] iceberg/conversion: json schema tests --- .../conversion/tests/iceberg_json_tests.cc | 79 +++++++++++++++++++ .../tests/datalake/datalake_e2e_test.py | 24 ++++++ 2 files changed, 103 insertions(+) diff --git a/src/v/iceberg/conversion/tests/iceberg_json_tests.cc b/src/v/iceberg/conversion/tests/iceberg_json_tests.cc index bcd7cdb8277c0..022e5691e262b 100644 --- a/src/v/iceberg/conversion/tests/iceberg_json_tests.cc +++ b/src/v/iceberg/conversion/tests/iceberg_json_tests.cc @@ -341,6 +341,48 @@ TEST(JsonSchema, PrimitiveTypesMixed) { result.error().what()); } +TEST(JsonSchema, ThreeWayAmbiguousUnion) { + constexpr std::string_view schema = R"({ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": ["boolean", "integer", "string"] + })"; + + auto result = to_iceberg_type(schema); + ASSERT_TRUE(result.has_error()); + ASSERT_STREQ( + "Type constraint is not sufficient for transforming. Types: [boolean, " + "integer, string]", + result.error().what()); +} + +TEST(JsonSchema, NullableAmbiguousUnion) { + constexpr std::string_view schema = R"({ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": ["null", "integer", "string"] + })"; + + auto result = to_iceberg_type(schema); + ASSERT_TRUE(result.has_error()); + ASSERT_STREQ( + "Type constraint is not sufficient for transforming. Types: [null, " + "integer, string]", + result.error().what()); +} + +TEST(JsonSchema, ContainerTypeUnion) { + constexpr std::string_view schema = R"({ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": ["object", "array"] + })"; + + auto result = to_iceberg_type(schema); + ASSERT_TRUE(result.has_error()); + ASSERT_STREQ( + "Type constraint is not sufficient for transforming. Types: [object, " + "array]", + result.error().what()); +} + TEST(JsonSchema, Nested) { auto result = to_iceberg_type(nested_schema); ASSERT_TRUE(result.has_value()) << result.error().what(); @@ -533,6 +575,24 @@ TEST(JsonSchema, ListWithItem) { iceberg::field_required::no)); } +TEST(JsonSchema, ListWithNullableItem) { + constexpr std::string_view schema = R"({ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "array", + "items": { "type": ["null", "string"] } + })"; + auto result = to_iceberg_type(schema); + ASSERT_TRUE(result.has_value()) << result.error().what(); + + ASSERT_EQ(result.value().fields.size(), 1); + ASSERT_TRUE(field_matches( + result.value().fields[0], + "root", + iceberg::list_type::create( + 0, iceberg::field_required::yes, iceberg::string_type{}), + iceberg::field_required::no)); +} + TEST(JsonSchema, ListWithEmptyItems) { constexpr std::string_view schema = R"({ "$schema": "http://json-schema.org/draft-07/schema#", @@ -750,6 +810,25 @@ TEST(JsonSchema, Format) { } } +TEST(JsonSchema, FormatIgnoredOnNonStringTypes) { + // Format annotations should be silently ignored on non-string types + // per JSON Schema spec (format is only meaningful for strings). + constexpr std::string_view schema = R"({ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "number", + "format": "date" + })"; + + auto result = to_iceberg_type(schema); + ASSERT_TRUE(result.has_value()) << result.error().what(); + ASSERT_EQ(result.value().fields.size(), 1); + ASSERT_TRUE(field_matches( + result.value().fields[0], + "root", + iceberg::double_type{}, + iceberg::field_required::no)); +} + TEST(JsonSchema, BannedKeywords) { for (const auto& keyword : { "patternProperties", diff --git a/tests/rptest/tests/datalake/datalake_e2e_test.py b/tests/rptest/tests/datalake/datalake_e2e_test.py index de9f338ba0d37..66b6a3d45cf6d 100644 --- a/tests/rptest/tests/datalake/datalake_e2e_test.py +++ b/tests/rptest/tests/datalake/datalake_e2e_test.py @@ -343,6 +343,30 @@ def generate_record(self, t): ("Part 0", "hours(redpanda.timestamp)", ""), ], ), + "array_items": JsonSchemaTestCase( + schema_str="""{ + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "properties": { + "numbers": { + "type": "array", + "items": {"type": ["null", "integer"]} + } + }, + "required": ["numbers"] + }""", + record_generator=lambda t: { + "numbers": [ + random.choice([None, int(t) + i]) for i in range(randint(0, 10)) + ] + }, + expected_spark=[ + ("numbers", "array", None), + ("", "", ""), + ("# Partitioning", "", ""), + ("Part 0", "hours(redpanda.timestamp)", ""), + ], + ), } JSON_SCHEMA_DLQ_TEST_CASES = { From 5d90791efeb263342342d54978d0ca02ba3e0c85 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Sat, 3 Jan 2026 13:19:14 +0000 Subject: [PATCH 2/2] iceberg/conversion: refactor JSON schema to constraint-based deduction Refactor the JSON Schema to Iceberg type conversion to use a two-phase constraint solver architecture instead of the previous mixed traversal/conversion approach. Phase 1 (collect): Traverses the JSON Schema and builds a constraint structure with a bitfield tracking possible types, format annotations, and nested constraints for objects/arrays. Phase 2 (resolve): Resolves constraints to Iceberg types, rejecting schemas where type cannot be unambiguously determined. This design provides a cleaner separation of concerns and is more extensible for future JSON Schema keywords (allOf, anyOf, oneOf, $ref). No functional changes intended. --- src/v/iceberg/conversion/BUILD | 1 + src/v/iceberg/conversion/ir_json.cc | 540 +++++++++++------- src/v/iceberg/conversion/json_schema/ir.h | 27 +- .../conversion/tests/iceberg_json_tests.cc | 20 +- 4 files changed, 365 insertions(+), 223 deletions(-) diff --git a/src/v/iceberg/conversion/BUILD b/src/v/iceberg/conversion/BUILD index b5f2858066fbf..3a2cc73f20532 100644 --- a/src/v/iceberg/conversion/BUILD +++ b/src/v/iceberg/conversion/BUILD @@ -170,6 +170,7 @@ redpanda_cc_library( "ir_json.h", ], implementation_deps = [ + "//src/v/base", "//src/v/iceberg:datatypes", "//src/v/iceberg/conversion/json_schema:frontend", "@fmt", diff --git a/src/v/iceberg/conversion/ir_json.cc b/src/v/iceberg/conversion/ir_json.cc index 14004952d6edb..5308251555bab 100644 --- a/src/v/iceberg/conversion/ir_json.cc +++ b/src/v/iceberg/conversion/ir_json.cc @@ -10,272 +10,384 @@ #include "iceberg/conversion/ir_json.h" +#include "base/format_to.h" +#include "base/vassert.h" #include "iceberg/conversion/conversion_outcome.h" #include "iceberg/conversion/json_schema/ir.h" #include "iceberg/datatypes.h" +#include #include -#include - +#include +#include +#include #include -#include +#include +namespace iceberg { namespace { -static constexpr iceberg::nested_field::id_t placeholder_field_id{0}; -} +constexpr iceberg::nested_field::id_t placeholder_field_id{0}; + +using json_type = conversion::json_schema::json_value_type; +using json_format = conversion::json_schema::format; + +/// Set of JSON value types for constraint tracking. +/// +/// Provides type-safe operations over a bitset where each bit corresponds +/// to a json_value_type enum value. +class type_set { +public: + /// Creates an unconstrained type set (all types allowed). + static type_set all() { + type_set s; + s.bits_.set(); + return s; + } -namespace iceberg { + /// Creates an empty type set (no types allowed). + static type_set none() { return type_set{}; } -namespace { + void set(json_type t) { bits_.set(index(t)); } + bool test(json_type t) const { return bits_.test(index(t)); } -std::optional -convert_type(const conversion::json_schema::json_value_type& t) { - switch (t) { - case conversion::json_schema::json_value_type::null: - return std::nullopt; - case conversion::json_schema::json_value_type::boolean: - return iceberg::boolean_type{}; - case conversion::json_schema::json_value_type::object: - return iceberg::struct_type{}; - case conversion::json_schema::json_value_type::array: - return iceberg::list_type{}; - case conversion::json_schema::json_value_type::number: - return iceberg::double_type{}; - case conversion::json_schema::json_value_type::integer: - return iceberg::long_type{}; - case conversion::json_schema::json_value_type::string: - return iceberg::string_type{}; + /// Returns the single non-null type if exactly one is set. + std::optional single_non_null_type() const { + auto copy = bits_; + copy.reset(index(json_type::null)); + if (copy.count() != 1) { + return std::nullopt; + } + for (auto t : conversion::json_schema::all_json_value_types) { + if (copy.test(index(t))) { + return t; + } + } + vunreachable("bitset count is 1 but no type found"); } - vunreachable("Unexpected JSON conversion type {}", t); -} - -conversion_outcome> -validation_types_to_field_type( - const std::vector& types) { - if (types.empty()) { - return std::nullopt; - } else if (types.size() == 1) { - return convert_type(types[0]); - } else if ( - types.size() == 2 - && types[0] == conversion::json_schema::json_value_type::null) { - return convert_type(types[1]); - } else if ( - types.size() == 2 - && types[1] == conversion::json_schema::json_value_type::null) { - return convert_type(types[0]); - } else { - return conversion_exception( - fmt::format( - "Type constraint is not sufficient for transforming. Types: [{}]", - fmt::join(types, ", "))); + /// Formats type names (for error messages). + fmt::iterator format_to(fmt::iterator it) const { + bool first = true; + for (auto t : conversion::json_schema::all_json_value_types) { + if (bits_.test(index(t))) { + if (!first) { + it = fmt::format_to(it, ", "); + } + it = fmt::format_to(it, "{}", t); + first = false; + } + } + return it; } -} -struct conversion_context { - conversion::json_schema::dialect dialect{}; +private: + using bits_t + = std::bitset; + + type_set() = default; - json_conversion_ir::struct_field_map_t field_index; + static size_t index(json_type t) { return static_cast(t); } - // Pointer to the current struct field index. We use it when walking nested - // structs. - json_conversion_ir::struct_field_map_t* current{&field_index}; + bits_t bits_; }; -conversion_outcome -convert(conversion_context& ctx, const conversion::json_schema::subschema& s) { - if (ctx.dialect != conversion::json_schema::dialect::draft7) { +/// Constraint collected from a JSON Schema. +/// +/// Models type deduction as constraint satisfaction: the types bitfield +/// tracks which JSON types are still possible. Resolution succeeds when +/// exactly one non-null type remains. +struct constraint { + constraint() = default; + constraint(constraint&&) = default; + constraint& operator=(constraint&&) = default; + constraint(const constraint&) = default; + constraint& operator=(const constraint&) = default; + ~constraint() = default; + + // Allowed types. Starts as all-set (unconstrained). + type_set types = type_set::all(); + + // Format annotation for strings (date, time, date-time). + std::optional format = std::nullopt; + + // Object properties, keyed by name. + std::map properties = {}; + + // Array item constraints. + // - nullopt: no "items" keyword + // - empty vector: "items": [] + // - non-empty: item schema(s) to validate + std::optional> items = std::nullopt; +}; + +/// Context for the resolution phase. +struct resolution_context { + json_conversion_ir::struct_field_map_t root_field_map; + json_conversion_ir::struct_field_map_t* current_field_map{&root_field_map}; +}; + +// Forward declarations. +conversion_outcome +collect(const conversion::json_schema::subschema&); + +conversion_outcome resolve(resolution_context&, const constraint&); + +/// Collect item constraints from a JSON Schema array. +conversion_outcome>> +collect_items(const conversion::json_schema::subschema& s) { + using ret_t = conversion_outcome>>; + + return ss::visit( + s.items(), + [](std::monostate) -> ret_t { return std::nullopt; }, + [](std::reference_wrapper item) + -> ret_t { + auto c = collect(item.get()); + if (c.has_error()) { + return c.error(); + } + return std::vector{std::move(c.value())}; + }, + [&s]( + const conversion::json_schema::const_list_view& tuple_items) -> ret_t { + std::vector result; + result.reserve(tuple_items.size() + (s.additional_items() ? 1 : 0)); + + for (const auto& item : tuple_items) { + auto c = collect(item); + if (c.has_error()) { + return c.error(); + } + result.push_back(std::move(c.value())); + } + + if (s.additional_items()) { + auto c = collect(s.additional_items()->get()); + if (c.has_error()) { + return c.error(); + } + result.push_back(std::move(c.value())); + } + + return result; + }); +} + +/// Collect constraints from a JSON Schema subschema. +conversion_outcome +collect(const conversion::json_schema::subschema& s) { + // Validate dialect for each subschema. + if (s.base().dialect() != conversion::json_schema::dialect::draft7) { return conversion_exception( - fmt::format("Unsupported JSON schema dialect: {}", ctx.dialect)); + fmt::format( + "Unsupported JSON schema dialect: {}", s.base().dialect())); } - std::optional t; + constraint c; + + // Type keyword. + const auto& schema_types = s.types(); + if (!schema_types.empty()) { + c.types = type_set::none(); + for (auto t : schema_types) { + c.types.set(t); + } + } - { - auto res = validation_types_to_field_type(s.types()); + // Format annotation. + c.format = s.format(); + + // Object properties (only if object type is possible). + if (c.types.test(json_type::object)) { + for (const auto& [name, prop] : s.properties()) { + auto prop_constraint = collect(prop); + if (prop_constraint.has_error()) { + return prop_constraint.error(); + } + c.properties[name] = std::move(prop_constraint.value()); + } - if (res.has_error()) { - return res.error(); - } else if (res.value().has_value()) { - t = std::move(*res.value()); + if ( + s.additional_properties() + && s.additional_properties()->get().boolean_subschema() != false) { + return conversion_exception( + "Only 'false' subschema is supported " + "for additionalProperties keyword"); } } - if (!t.has_value()) { - return conversion_exception( - fmt::format("Unsupported JSON conversion: missing type keyword")); + // Array items (only if array type is possible). + if (c.types.test(json_type::array)) { + auto items_result = collect_items(s); + if (items_result.has_error()) { + return items_result.error(); + } + c.items = std::move(items_result.value()); } - if (*t == iceberg::string_type{} && s.format().has_value()) { - // If the type is string, we can have a format. - switch (s.format().value()) { - case conversion::json_schema::format::date_time: - return iceberg::timestamptz_type{}; - case conversion::json_schema::format::date: - return iceberg::date_type{}; - case conversion::json_schema::format::time: - return iceberg::time_type{}; + return c; +} + +/// Resolve object constraint to Iceberg struct. +conversion_outcome +resolve_object(resolution_context& ctx, const constraint& c) { + struct_type result; + + // std::map iterates in sorted key order, giving deterministic field + // ordering. + for (const auto& [name, prop] : c.properties) { + // Recurse with a fresh field map for nested structs. + json_conversion_ir::struct_field_map_t nested_map; + + // Scope defer to cover only the recursive call - restore must happen + // before we emplace into the parent's field map. + std::optional resolved_type; + { + auto* prev_field_map = std::exchange( + ctx.current_field_map, &nested_map); + auto restore = ss::defer([&ctx, prev_field_map] noexcept { + ctx.current_field_map = prev_field_map; + }); + + auto resolved = resolve(ctx, prop); + if (resolved.has_error()) { + return resolved.error(); + } + resolved_type = std::move(resolved.value()); } - return std::move(*t); - } + // Record field position for value deserialization. + auto pos = result.fields.size(); + auto [_, inserted] = ctx.current_field_map->emplace( + name, + json_conversion_ir::field_annotation{ + .field_pos = pos, .nested_fields = std::move(nested_map)}); + if (!inserted) { + return conversion_exception( + fmt::format("Duplicate field name in JSON schema: {}", name)); + } - return ss::visit( - *t, - [](const iceberg::primitive_type& t) - -> conversion_outcome { return t; }, - [&ctx, &s]( - iceberg::struct_type& st) -> conversion_outcome { - auto sorted_prop_keys = std::views::keys(s.properties()) - | std::ranges::to>(); - std::ranges::sort(sorted_prop_keys); - for (const auto& name : sorted_prop_keys) { - auto field_index = json_conversion_ir::struct_field_map_t{}; - - // Depth first. - auto tmp = ctx.current; - ctx.current = &field_index; - auto child_res = convert(ctx, s.properties().at(name)); - if (child_res.has_error()) { - return child_res.error(); - } + result.fields.push_back( + nested_field::create( + placeholder_field_id, + name, + field_required::no, + std::move(*resolved_type))); + } - auto field_position = st.fields.size(); - - // After we converted the children, restore the context and - // update it. - ctx.current = tmp; - if (!ctx.current - ->emplace( - name, - json_conversion_ir::field_annotation{ - field_position, std::move(field_index)}) - .second) { - return conversion_exception( - fmt::format( - "Duplicate field name in JSON schema: {}", name)); - } + return result; +} - st.fields.push_back( - iceberg::nested_field::create( - placeholder_field_id, - name, - iceberg::field_required::no, - std::move(child_res.value()))); - } +/// Resolve array constraint to Iceberg list. +conversion_outcome +resolve_array(resolution_context& ctx, const constraint& c) { + if (!c.items.has_value()) { + return conversion_exception( + "Cannot convert JSON schema list type without items"); + } - if ( - s.additional_properties() - && s.additional_properties()->get().boolean_subschema() != false) { - return conversion_exception( - "Only 'false' subschema is supported " - "for additionalProperties keyword"); - } + if (c.items->empty()) { + return conversion_exception( + "List type items must have the type defined in JSON schema"); + } - return std::move(st); - }, - [&ctx, &s]( - const iceberg::list_type&) -> conversion_outcome { - using ret_t = conversion_outcome; - - return ss::visit( - s.items(), - [](const std::monostate&) -> ret_t { - return conversion_exception( - "Cannot convert JSON schema list type without items"); - }, - [&]( - const std::reference_wrapper< - const conversion::json_schema::subschema>& item) -> ret_t { - auto item_res = convert(ctx, item.get()); - if (item_res.has_error()) { - return item_res.error(); - } + // Resolve all item schemas and verify they produce the same type. + std::optional element_type; + for (const auto& item : *c.items) { + auto resolved = resolve(ctx, item); + if (resolved.has_error()) { + return resolved.error(); + } - return iceberg::list_type::create( - placeholder_field_id, - iceberg::field_required::yes, - std::move(item_res.value())); - }, - [&](const iceberg::conversion::json_schema::const_list_view& items) - -> ret_t { - std::optional resolved_type; - - for (const auto& item : items) { - auto item_res = convert(ctx, item); - if (item_res.has_error()) { - return item_res.error(); - } - - if (!resolved_type.has_value()) { - resolved_type = std::move(item_res.value()); - } else if (*resolved_type != item_res.value()) { - return conversion_exception( - fmt::format( - "List type items must have the same type, but " - "found " - "{} and {}", - *resolved_type, - item_res.value())); - } - } + if (!element_type) { + element_type = std::move(resolved.value()); + } else if (*element_type != resolved.value()) { + return conversion_exception( + fmt::format( + "List type items must have the same type, but found {} and {}", + *element_type, + resolved.value())); + } + } - if (s.additional_items()) { - auto additional_item_res = convert( - ctx, s.additional_items().value().get()); - if (additional_item_res.has_error()) { - return additional_item_res.error(); - } - if (!resolved_type.has_value()) { - resolved_type = std::move(additional_item_res.value()); - } else if (*resolved_type != additional_item_res.value()) { - return conversion_exception( - fmt::format( - "List type items must have the same type, but " - "found " - "{} " - "and {}", - *resolved_type, - additional_item_res.value())); - } - } + return list_type::create( + placeholder_field_id, field_required::yes, std::move(*element_type)); +} - if (!resolved_type.has_value()) { - return conversion_exception( - "List type items must have the type defined in JSON " - "schema"); - } +/// Resolve a constraint to an Iceberg field type. +conversion_outcome +resolve(resolution_context& ctx, const constraint& c) { + auto single_type = c.types.single_non_null_type(); + if (!single_type) { + return conversion_exception( + fmt::format( + "Type constraint is not sufficient for transforming. Types: [{}]", + c.types)); + } - return iceberg::list_type::create( - placeholder_field_id, - iceberg::field_required::yes, - std::move(*resolved_type)); - }); - }, - [](iceberg::map_type&) -> conversion_outcome { - return conversion_exception( - "Map type is not expected in JSON schema conversion"); - }); + switch (*single_type) { + case json_type::boolean: + return boolean_type{}; + case json_type::integer: + return long_type{}; + case json_type::number: + return double_type{}; + case json_type::string: + if (c.format) { + switch (*c.format) { + case json_format::date_time: + return timestamptz_type{}; + case json_format::date: + return date_type{}; + case json_format::time: + return time_type{}; + } + } + return string_type{}; + case json_type::object: { + auto result = resolve_object(ctx, c); + if (result.has_error()) { + return result.error(); + } + return std::move(result.value()); + } + case json_type::array: { + auto result = resolve_array(ctx, c); + if (result.has_error()) { + return result.error(); + } + return std::move(result.value()); + } + case json_type::null: + vunreachable("null type should not be resolved"); + } } } // namespace conversion_outcome type_to_ir(const conversion::json_schema::schema& schema) { - conversion_context ctx; + if (schema.root().dialect() != conversion::json_schema::dialect::draft7) { + return conversion_exception( + fmt::format( + "Unsupported JSON schema dialect: {}", schema.root().dialect())); + } - ctx.dialect = schema.root().dialect(); + auto c = collect(schema.root()); + if (c.has_error()) { + return c.error(); + } - auto r = convert(ctx, schema.root()); - if (r.has_error()) { - return r.error(); + resolution_context ctx; + auto result = resolve(ctx, c.value()); + if (result.has_error()) { + return result.error(); } return json_conversion_ir( - std::make_unique(std::move(r.value())), - ctx.field_index); + std::make_unique(std::move(result.value())), + std::move(ctx.root_field_map)); } } // namespace iceberg diff --git a/src/v/iceberg/conversion/json_schema/ir.h b/src/v/iceberg/conversion/json_schema/ir.h index d0fcde3edb182..9219b53410779 100644 --- a/src/v/iceberg/conversion/json_schema/ir.h +++ b/src/v/iceberg/conversion/json_schema/ir.h @@ -57,8 +57,8 @@ constexpr auto format_by_name = std::to_array({ /// Primitive data types. /// https://json-schema.org/draft/2020-12/json-schema-core#name-instance-data-model -enum class json_value_type { - null, +enum class json_value_type : uint8_t { + null = 0, boolean, object, array, @@ -67,6 +67,29 @@ enum class json_value_type { string, }; +constexpr auto all_json_value_types = std::to_array({ + json_value_type::null, + json_value_type::boolean, + json_value_type::object, + json_value_type::array, + json_value_type::number, + json_value_type::integer, + json_value_type::string, +}); + +consteval bool json_value_types_contiguous() { + for (size_t i = 0; i < all_json_value_types.size(); ++i) { + if (static_cast(all_json_value_types[i]) != i) { + return false; + } + } + return true; +} + +static_assert( + json_value_types_contiguous(), + "json_value_type order/values drifted from all_json_value_types"); + json_value_type parse_json_value_type(const std::string&); class subschema; diff --git a/src/v/iceberg/conversion/tests/iceberg_json_tests.cc b/src/v/iceberg/conversion/tests/iceberg_json_tests.cc index 022e5691e262b..d5326b168cfc5 100644 --- a/src/v/iceberg/conversion/tests/iceberg_json_tests.cc +++ b/src/v/iceberg/conversion/tests/iceberg_json_tests.cc @@ -291,7 +291,8 @@ TEST(JsonSchema, Empty) { auto result = to_iceberg_type(schema_str); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); } } @@ -306,7 +307,7 @@ TEST(JsonSchema, NullType) { auto result = to_iceberg_type(schema_str); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null]", result.error().what()); } } @@ -478,7 +479,8 @@ TEST(JsonSchema, ObjectWithInvalidProperty) { auto result = to_iceberg_type(schema); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); } @@ -511,7 +513,8 @@ TEST(JsonSchema, ObjectWithBooleanProperty) { auto result = to_iceberg_type(schema); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); } @@ -538,7 +541,8 @@ TEST(JsonSchema, ListWithInvalidItems) { auto result = to_iceberg_type(schema); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); } @@ -552,7 +556,8 @@ TEST(JsonSchema, ListWithInvalidItemsList) { auto result = to_iceberg_type(schema); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); } @@ -703,7 +708,8 @@ TEST(JsonSchema, ListWithItemAndInvalidAdditionalItems) { auto result = to_iceberg_type(schema); ASSERT_TRUE(result.has_error()); ASSERT_STREQ( - "Unsupported JSON conversion: missing type keyword", + "Type constraint is not sufficient for transforming. Types: [null, " + "boolean, object, array, number, integer, string]", result.error().what()); }