mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
test_for_basic_auth_registry - mock up
This commit is contained in:
parent
46e03432a9
commit
8b0fc82834
@ -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
|
||||
|
@ -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,
|
||||
)
|
||||
|
@ -0,0 +1 @@
|
||||
schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user
|
@ -0,0 +1,5 @@
|
||||
RealmFooBar {
|
||||
org.eclipse.jetty.jaas.spi.PropertyFileLoginModule required
|
||||
file="/etc/schema-registry/secrets/password"
|
||||
debug="true";
|
||||
};
|
@ -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
|
||||
)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user