Skip to content

Commit

Permalink
Merge pull request redpanda-data#11434 from BenPope/server-side-schem…
Browse files Browse the repository at this point in the history
…a-id-validation-config-check

schema_registry: Schema id validation - config checking
  • Loading branch information
BenPope committed Jun 14, 2023
2 parents 0ad8dd3 + 0891042 commit e77e852
Show file tree
Hide file tree
Showing 3 changed files with 85 additions and 5 deletions.
13 changes: 10 additions & 3 deletions src/v/pandaproxy/schema_registry/validation.cc
Original file line number Diff line number Diff line change
Expand Up @@ -457,9 +457,16 @@ std::optional<schema_id_validator> 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<schema_id_validator>(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<schema_id_validator>(api, topic, props, mode);
}
return std::nullopt;
}

ss::future<schema_id_validator::result>
Expand Down
13 changes: 12 additions & 1 deletion src/v/redpanda/admin_server.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
Expand Down Expand Up @@ -1244,6 +1246,7 @@ join_properties(const std::vector<std::reference_wrapper<
*/
void config_multi_property_validation(
ss::sstring const& username,
pandaproxy::schema_registry::api* schema_registry,
cluster::config_update_request const& req,
config::configuration const& updated_config,
std::map<ss::sstring, ss::sstring>& errors) {
Expand Down Expand Up @@ -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() {
Expand Down Expand Up @@ -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;
Expand Down
64 changes: 63 additions & 1 deletion tests/rptest/tests/schema_registry_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
import json
from typing import Optional
import uuid
import re
import requests
import time
import random
Expand All @@ -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
Expand Down Expand Up @@ -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,
Expand All @@ -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)
Expand Down Expand Up @@ -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':
Expand Down

0 comments on commit e77e852

Please sign in to comment.