Skip to content

Commit

Permalink
sr/test/json: many-error compat smoketest
Browse files Browse the repository at this point in the history
Add a smoketest to further test that multiple (as many as possible)
errors are reported for a complex schema. This is checked in both the
forward and backward direction.
  • Loading branch information
pgellert committed Sep 10, 2024
1 parent 337e067 commit ca18cfb
Showing 1 changed file with 108 additions and 0 deletions.
108 changes: 108 additions & 0 deletions src/v/pandaproxy/schema_registry/test/test_json_schema.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
#include <seastar/testing/thread_test_case.hh>
#include <seastar/util/defer.hh>

#include <absl/container/flat_hash_set.h>
#include <boost/test/tools/context.hpp>
#include <fmt/core.h>

Expand All @@ -36,6 +37,20 @@ bool check_compatible(
return res.is_compat;
}

pps::compatibility_result check_compatible_verbose(
const pps::canonical_schema_definition& r,
const pps::canonical_schema_definition& w) {
pps::sharded_store s;
return check_compatible(
pps::make_json_schema_definition(
s, {pps::subject("r"), {r.shared_raw(), pps::schema_type::json}})
.get(),
pps::make_json_schema_definition(
s, {pps::subject("w"), {w.shared_raw(), pps::schema_type::json}})
.get(),
pps::verbose::yes);
}

struct store_fixture {
store_fixture() {
store.start(pps::is_mutable::yes, ss::default_smp_service_group())
Expand Down Expand Up @@ -1449,3 +1464,96 @@ SEASTAR_THREAD_TEST_CASE(test_compatibility_check) {
}
};
}

namespace {

const auto schema_old = pps::canonical_schema_definition({
R"(
{
"type": "object",
"minProperties": 2,
"maxProperties": 9,
"properties": {
"aaaa": {"type": "integer"},
"bbbb": {"type": "string"},
"cccc": {"type": "boolean"}
},
"patternProperties": {
"^b": {"type": "string", "minLength":10}
},
"additionalProperties": false,
"required": ["aaaa", "cccc"],
"dependencies": {"a": ["b", "c", "d"], "b": {"type": "integer"}, "d": ["a"]}
})",
pps::schema_type::json});

const auto schema_new = pps::canonical_schema_definition({
R"(
{
"type": "object",
"minProperties": 3,
"maxProperties": 10,
"properties": {
"aaaa": {"type": "number"}
},
"patternProperties": {
"^b": {"type": "string"}
},
"additionalProperties": {"type": "boolean"},
"required": ["aaaa"],
"dependencies": {"a":["c", "b"], "b": {"type": "number"}}
})",
pps::schema_type::json});

const absl::flat_hash_set<incompatibility> forward_expected{
{"#/properties/aaaa", incompat_t::type_narrowed},
// TODO: implement the check for
// required_property_added_to_unopen_content_model. That is also expected
// for this schema update.
// {"#/properties/cccc",
// incompat_t::required_property_added_to_unopen_content_model},
{"#/dependencies/a", incompat_t::dependency_array_extended},
{"#/dependencies/d", incompat_t::dependency_array_added},
{"#/dependencies/b", incompat_t::type_narrowed},
{"#/additionalProperties", incompat_t::additional_properties_removed},
{"#/maxProperties", incompat_t::max_properties_decreased},
};
const absl::flat_hash_set<incompatibility> backward_expected{
{"#/minProperties", incompat_t::min_properties_increased},
};

const auto compat_data = std::to_array<compat_test_data<incompatibility>>({
{
schema_old.share(),
schema_new.share(),
forward_expected,
},
{
schema_new.share(),
schema_old.share(),
backward_expected,
},
});

std::string format_set(const absl::flat_hash_set<ss::sstring>& d) {
return fmt::format("{}", fmt::join(d, "\n"));
}

} // namespace

SEASTAR_THREAD_TEST_CASE(test_json_compat_messages) {
for (const auto& cd : compat_data) {
auto compat = check_compatible_verbose(cd.reader, cd.writer);

absl::flat_hash_set<ss::sstring> errs{
compat.messages.begin(), compat.messages.end()};
absl::flat_hash_set<ss::sstring> expected{
cd.expected.messages.begin(), cd.expected.messages.end()};

BOOST_CHECK(!compat.is_compat);
BOOST_CHECK_EQUAL(errs.size(), expected.size());
BOOST_REQUIRE_MESSAGE(
errs == expected,
fmt::format("{} != {}", format_set(errs), format_set(expected)));
}
}

0 comments on commit ca18cfb

Please sign in to comment.