diff --git a/src/v/pandaproxy/schema_registry/validation.cc b/src/v/pandaproxy/schema_registry/validation.cc index d5ef90881c65..c2eef9d1b404 100644 --- a/src/v/pandaproxy/schema_registry/validation.cc +++ b/src/v/pandaproxy/schema_registry/validation.cc @@ -457,9 +457,16 @@ std::optional maybe_make_schema_id_validator( const model::topic& topic, const cluster::topic_properties& props) { auto mode = config::shard_local_cfg().enable_schema_id_validation(); - return api != nullptr && should_validate_schema_id(props, mode) - ? std::make_optional(api, topic, props, mode) - : std::nullopt; + if (should_validate_schema_id(props, mode)) { + if (!api) { + vlog( + plog.error, + "{} requires schema_registry to be enabled in redpanda.yaml", + config::shard_local_cfg().enable_schema_id_validation.name()); + } + return std::make_optional(api, topic, props, mode); + } + return std::nullopt; } ss::future diff --git a/src/v/redpanda/admin_server.cc b/src/v/redpanda/admin_server.cc index f0b0e36c99b9..8e1449c389f3 100644 --- a/src/v/redpanda/admin_server.cc +++ b/src/v/redpanda/admin_server.cc @@ -63,6 +63,7 @@ #include "net/dns.h" #include "pandaproxy/rest/api.h" #include "pandaproxy/schema_registry/api.h" +#include "pandaproxy/schema_registry/schema_id_validation.h" #include "raft/types.h" #include "redpanda/admin/api-doc/broker.json.h" #include "redpanda/admin/api-doc/cluster.json.h" @@ -86,6 +87,7 @@ #include "security/scram_credential.h" #include "ssx/future-util.h" #include "ssx/metrics.h" +#include "ssx/sformat.h" #include "utils/fragmented_vector.h" #include "utils/string_switch.h" #include "utils/utf8.h" @@ -1244,6 +1246,7 @@ join_properties(const std::vector& errors) { @@ -1337,6 +1340,14 @@ void config_multi_property_validation( } } } + if ( + updated_config.enable_schema_id_validation + != pandaproxy::schema_registry::schema_id_validation_mode::none + && !schema_registry) { + auto name = updated_config.enable_schema_id_validation.name(); + errors[ss::sstring(name)] = ssx::sformat( + "{} requires schema_registry to be enabled in redpanda.yaml", name); + } } void admin_server::register_cluster_config_routes() { @@ -1542,7 +1553,7 @@ admin_server::patch_cluster_config_handler( // After checking each individual property, check for // any multi-property validation errors config_multi_property_validation( - auth_state.get_username(), update, cfg, errors); + auth_state.get_username(), _schema_registry, update, cfg, errors); if (!errors.empty()) { json::StringBuffer buf; diff --git a/tests/rptest/tests/schema_registry_test.py b/tests/rptest/tests/schema_registry_test.py index b9b78275074e..7fac42f17868 100644 --- a/tests/rptest/tests/schema_registry_test.py +++ b/tests/rptest/tests/schema_registry_test.py @@ -12,6 +12,7 @@ import json from typing import Optional import uuid +import re import requests import time import random @@ -30,7 +31,7 @@ from rptest.services import tls from rptest.services.admin import Admin from rptest.services.cluster import cluster -from rptest.services.redpanda import ResourceSettings, SecurityConfig, LoggingConfig, PandaproxyConfig, SchemaRegistryConfig +from rptest.services.redpanda import DEFAULT_LOG_ALLOW_LIST, ResourceSettings, SecurityConfig, LoggingConfig, PandaproxyConfig, SchemaRegistryConfig from rptest.services.serde_client import SerdeClient from rptest.tests.cluster_config_test import wait_for_version_status_sync from rptest.tests.pandaproxy_test import User, PandaProxyTLSProvider @@ -2156,6 +2157,65 @@ def test_mtls_and_basic_auth(self): assert set(result) == {"PROTOBUF", "AVRO"} +class SchemaValidationEnableWithoutSchemaRegistry(RedpandaTest): + def __init__(self, *args, **kwargs): + super(SchemaValidationEnableWithoutSchemaRegistry, + self).__init__(*args, schema_registry_config=None, **kwargs) + self.rpk = RpkTool(self.redpanda) + self.admin = Admin(self.redpanda) + + @cluster(num_nodes=1) + @parametrize(mode=SchemaIdValidationMode.REDPANDA) + @parametrize(mode=SchemaIdValidationMode.COMPAT) + def test_enable_schema_id_validation(self, mode): + try: + self.redpanda.set_cluster_config( + {'enable_schema_id_validation': mode}) + assert False, "expected failure" + except requests.exceptions.HTTPError as ex: + print(ex) + pass + + +class SchemaValidationWithoutSchemaRegistry(RedpandaTest): + INVALID_CONFIG_LOG_ALLOW_LIST = DEFAULT_LOG_ALLOW_LIST + [ + re.compile( + r"enable_schema_id_validation requires schema_registry to be enabled in redpanda.yaml" + ), + ] + + def __init__(self, *args, **kwargs): + super(SchemaValidationWithoutSchemaRegistry, + self).__init__(*args, + extra_rp_conf={ + 'enable_schema_id_validation': + SchemaIdValidationMode.REDPANDA.value + }, + schema_registry_config=None, + **kwargs) + + @cluster(num_nodes=1, log_allow_list=INVALID_CONFIG_LOG_ALLOW_LIST) + def test_disabled_schema_registry(self): + rpk = RpkTool(self.redpanda) + topic = "no_schema_registry" + rpk.create_topic( + topic, + config={ + TopicSpec.PROPERTY_RECORD_KEY_SCHEMA_ID_VALIDATION: 'true', + }) + try: + rpk.produce(topic, "key", "value") + assert False, "expected INVALID_RECORD" + except RpkException as e: + print(e) + assert "INVALID_RECORD" in e.stderr + + wait_until(lambda: self.redpanda.search_log_all( + "enable_schema_id_validation requires schema_registry to be enabled in redpanda.yaml" + ), + timeout_sec=5) + + class SchemaValidationTopicPropertiesTest(RedpandaTest): def __init__(self, *args, **kwargs): super(SchemaValidationTopicPropertiesTest, @@ -2164,6 +2224,7 @@ def __init__(self, *args, **kwargs): 'enable_schema_id_validation': SchemaIdValidationMode.COMPAT.value }, + schema_registry_config=SchemaRegistryConfig(), **kwargs) self.rpk = RpkTool(self.redpanda) self.admin = Admin(self.redpanda) @@ -2352,6 +2413,7 @@ def __init__(self, *args, **kwargs): 'enable_schema_id_validation': SchemaIdValidationMode.NONE.value }, + schema_registry_config=SchemaRegistryConfig(), **kwargs) self.redpanda.set_environment({ '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC':