test_for_basic_auth_registry - mock up

This commit is contained in:
Ilya Golshtein 2023-04-29 21:23:55 +00:00
parent 46e03432a9
commit 8b0fc82834
5 changed files with 43 additions and 13 deletions

View File

@ -39,7 +39,17 @@ services:
environment:
SCHEMA_REGISTRY_HOST_NAME: schema-registry
SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT
# SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC
SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092
SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC
# SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO
# SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein
# SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein
SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser
SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar
SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf"
volumes:
- ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets
depends_on:
- kafka_zookeeper
- kafka1

View File

@ -1159,10 +1159,13 @@ class ClickHouseCluster:
]
return self.base_kerberized_hdfs_cmd
def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir):
def setup_kafka_cmd(
self, instance, env_variables, docker_compose_yml_dir
):
self.with_kafka = True
env_variables["KAFKA_HOST"] = self.kafka_host
env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port)
env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/"
env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port)
env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081"
self.base_cmd.extend(
@ -1498,6 +1501,7 @@ class ClickHouseCluster:
with_kafka=False,
with_kerberized_kafka=False,
with_kerberos_kdc=False,
with_secrets=False,
with_rabbitmq=False,
with_nats=False,
clickhouse_path_dir=None,
@ -1604,6 +1608,7 @@ class ClickHouseCluster:
with_nats=with_nats,
with_nginx=with_nginx,
with_kerberized_hdfs=with_kerberized_hdfs,
with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka,
with_mongo=with_mongo or with_mongo_secure,
with_meili=with_meili,
with_redis=with_redis,
@ -3135,6 +3140,7 @@ class ClickHouseInstance:
with_nats,
with_nginx,
with_kerberized_hdfs,
with_secrets,
with_mongo,
with_meili,
with_redis,
@ -3197,7 +3203,7 @@ class ClickHouseInstance:
if clickhouse_path_dir
else None
)
self.kerberos_secrets_dir = p.abspath(p.join(base_path, "secrets"))
self.secrets_dir = p.abspath(p.join(base_path, "secrets"))
self.macros = macros if macros is not None else {}
self.with_zookeeper = with_zookeeper
self.zookeeper_config_path = zookeeper_config_path
@ -3220,6 +3226,7 @@ class ClickHouseInstance:
self.with_nats = with_nats
self.with_nginx = with_nginx
self.with_kerberized_hdfs = with_kerberized_hdfs
self.with_secrets = with_secrets
self.with_mongo = with_mongo
self.with_meili = with_meili
self.with_redis = with_redis
@ -4217,17 +4224,16 @@ class ClickHouseInstance:
if self.with_zookeeper:
shutil.copy(self.zookeeper_config_path, conf_d_dir)
if (
self.with_kerberized_kafka
or self.with_kerberized_hdfs
or self.with_kerberos_kdc
):
if self.with_secrets:
if self.with_kerberos_kdc:
base_secrets_dir = self.cluster.instances_dir
else:
base_secrets_dir = self.path
from_dir=self.secrets_dir
to_dir=p.abspath(p.join(base_secrets_dir, "secrets"))
logging.debug(f"Copy secret from {from_dir} to {to_dir}")
shutil.copytree(
self.kerberos_secrets_dir,
self.secrets_dir,
p.abspath(p.join(base_secrets_dir, "secrets")),
dirs_exist_ok=True,
)

View File

@ -0,0 +1 @@
schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user

View File

@ -0,0 +1,5 @@
RealmFooBar {
org.eclipse.jetty.jaas.spi.PropertyFileLoginModule required
file="/etc/schema-registry/secrets/password"
debug="true";
};

View File

@ -14,7 +14,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance
def started_cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance("dummy", with_kafka=True)
cluster.add_instance("dummy", with_kafka=True, with_secrets=True)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
@ -39,10 +39,13 @@ def run_query(instance, query, data=None, settings=None):
def test_select(started_cluster):
# type: (ClickHouseCluster) -> None
input("Cluster created, press any key to destroy...")
schema_registry_client = CachedSchemaRegistryClient(
"http://localhost:{}".format(started_cluster.schema_registry_port)
)
reg_url="http://localhost:{}".format(
started_cluster.schema_registry_port)
arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'}
schema_registry_client = CachedSchemaRegistryClient(arg)
serializer = MessageSerializer(schema_registry_client)
schema = avro.schema.make_avsc_object(
@ -62,7 +65,12 @@ def test_select(started_cluster):
data = buf.getvalue()
instance = started_cluster.instances["dummy"] # type: ClickHouseInstance
schema_registry_url = "http://{}:{}".format(
# schema_registry_url = "http://{}:{}@{}:{}".format(
# 'schemauser', 'letmein',
# started_cluster.schema_registry_host, 8081
# )
schema_registry_url = "http://{}:{}@{}:{}".format(
'schemauser', 'letmein',
started_cluster.schema_registry_host, 8081
)