From c12fa26fe344934aafc4eb8c2f6e43501385ec6b Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 23 Aug 2020 23:59:21 +0300 Subject: [PATCH 01/15] kerberized kafka test works --- .../docker_compose_kerberized_kafka.yml | 56 +++++++ tests/integration/helpers/cluster.py | 46 +++++- tests/integration/tskk/Dockerfile | 11 ++ tests/integration/tskk/__init__.py | 0 .../format_schemas/kafka.proto | 6 + .../format_schemas/template_row.format | 1 + .../clickhouse_path/format_schemas/test.avsc | 11 ++ .../clickhouse_path/format_schemas/test.capnp | 10 ++ .../clickhouse_path/format_schemas/test.proto | 9 ++ tests/integration/tskk/configs/kafka.xml | 26 ++++ tests/integration/tskk/configs/log_conf.xml | 11 ++ tests/integration/tskk/configs/users.xml | 25 ++++ tests/integration/tskk/kafka_pb2.py | 76 ++++++++++ .../integration/tskk/kerberos_image_config.sh | 137 ++++++++++++++++++ .../integration/tskk/secrets/broker_jaas.conf | 15 ++ tests/integration/tskk/secrets/krb.conf | 22 +++ .../tskk/secrets/zookeeper_jaas.conf | 14 ++ tests/integration/tskk/test.py | 137 ++++++++++++++++++ .../tskk/test_kafka_json.reference | 50 +++++++ .../tskk/test_kafka_virtual1.reference | 50 +++++++ .../tskk/test_kafka_virtual2.reference | 50 +++++++ 21 files changed, 761 insertions(+), 2 deletions(-) create mode 100644 docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml create mode 100644 tests/integration/tskk/Dockerfile create mode 100644 tests/integration/tskk/__init__.py create mode 100755 tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto create mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/template_row.format create mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.avsc create mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.capnp create mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.proto create mode 100644 tests/integration/tskk/configs/kafka.xml create mode 100644 tests/integration/tskk/configs/log_conf.xml create mode 100644 tests/integration/tskk/configs/users.xml create mode 100644 tests/integration/tskk/kafka_pb2.py create mode 100644 tests/integration/tskk/kerberos_image_config.sh create mode 100644 tests/integration/tskk/secrets/broker_jaas.conf create mode 100644 tests/integration/tskk/secrets/krb.conf create mode 100644 tests/integration/tskk/secrets/zookeeper_jaas.conf create mode 100644 tests/integration/tskk/test.py create mode 100644 tests/integration/tskk/test_kafka_json.reference create mode 100644 tests/integration/tskk/test_kafka_virtual1.reference create mode 100644 tests/integration/tskk/test_kafka_virtual2.reference diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml new file mode 100644 index 0000000000..f06e26c76c --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -0,0 +1,56 @@ +version: '2.3' + +services: + kafka_kerberized_zookeeper: + image: confluentinc/cp-zookeeper:5.2.0 + # restart: always + hostname: kafka_kerberized_zookeeper + environment: + ZOOKEEPER_SERVER_ID: 1 + ZOOKEEPER_CLIENT_PORT: 2181 + ZOOKEEPER_SERVERS: "kafka_kerberized_zookeeper:2888:3888" + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/zookeeper_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider -Dsun.security.krb5.debug=true" + volumes: + - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/etc/kafka/secrets + - /dev/urandom:/dev/random + depends_on: + - kafka_kerberos + security_opt: + - label:disable + + kerberized_kafka1: + image: confluentinc/cp-kafka:5.2.0 + # restart: always + hostname: kerberized_kafka1 + ports: + - "9092:9092" + - "9093:9093" + environment: + KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 + KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 + # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 + # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 + KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI + KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI + KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: OUTSIDE:SASL_PLAINTEXT,UNSECURED_OUTSIDE:PLAINTEXT,UNSECURED_INSIDE:PLAINTEXT, + KAFKA_INTER_BROKER_LISTENER_NAME: OUTSIDE + KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: "kafka_kerberized_zookeeper:2181" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true" + volumes: + - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/etc/kafka/secrets + depends_on: + - kafka_kerberized_zookeeper + - kafka_kerberos + security_opt: + - label:disable + + kafka_kerberos: + build: /home/ilejn/projects/ClickHouse/tests/integration/tskk + hostname: kafka_kerberos + volumes: + - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/tmp/keytab + ports: [88, 749] diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f5d9290a17..ab038bc702 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -125,6 +125,7 @@ class ClickHouseCluster: self.base_zookeeper_cmd = None self.base_mysql_cmd = [] self.base_kafka_cmd = [] + self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] self.base_cassandra_cmd = [] self.pre_zookeeper_commands = [] @@ -133,6 +134,7 @@ class ClickHouseCluster: self.with_mysql = False self.with_postgres = False self.with_kafka = False + self.with_kerberized_kafka = False self.with_rabbitmq = False self.with_odbc_drivers = False self.with_hdfs = False @@ -169,7 +171,7 @@ class ClickHouseCluster: def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, + with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -207,6 +209,7 @@ class ClickHouseCluster: zookeeper_config_path=self.zookeeper_config_path, with_mysql=with_mysql, with_kafka=with_kafka, + with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, with_mongo=with_mongo, with_redis=with_redis, @@ -290,6 +293,13 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) + if with_kerberized_kafka and not self.with_kerberized_kafka: + self.with_kerberized_kafka = True + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) + self.base_kerberized_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] + cmds.append(self.base_kerberized_kafka_cmd) + if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) @@ -608,6 +618,13 @@ class ClickHouseCluster: self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.wait_schema_registry_to_start(120) + if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: + env = os.environ.copy() + self.kerberized_kafka_instance_path = instance.path + env['KERBERIZED_KAFKA_DIR'] = self.kerberized_kafka_instance_path + '/' + subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) + self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') + if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') @@ -778,9 +795,12 @@ services: - {instance_config_dir}:/etc/clickhouse-server/ - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ + - /etc/passwd:/etc/passwd:ro {binary_volume} {odbc_bridge_volume} {odbc_ini_path} + {keytab_path} + {krb5_conf} entrypoint: {entrypoint_cmd} tmpfs: {tmpfs} cap_add: @@ -810,7 +830,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -838,6 +858,7 @@ class ClickHouseInstance: self.with_mysql = with_mysql self.with_kafka = with_kafka + self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq self.with_mongo = with_mongo self.with_redis = with_redis @@ -853,6 +874,13 @@ class ClickHouseInstance: else: self.odbc_ini_path = "" + if with_kerberized_kafka: + self.keytab_path = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets:/tmp/keytab" + self.krb5_conf = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets/krb.conf:/etc/krb5.conf:ro" + else: + self.keytab_path = "" + self.krb5_conf = "" + self.docker_client = None self.ip_address = None self.client = None @@ -1182,6 +1210,15 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) + if self.with_kerberized_kafka: + # shutil.copytree(p.abspath(p.join(self.base_dir, 'secrets')), p.abspath(p.join(self.path, 'secrets'))) + secrets_dir = p.abspath(p.join(self.custom_config_dir, os.pardir, 'secrets')) + distutils.dir_util.copy_tree(secrets_dir, p.abspath(p.join(self.path, 'secrets'))) + + # Copy config dir + if self.custom_config_dir: + distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir) + # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: @@ -1217,6 +1254,9 @@ class ClickHouseInstance: depends_on.append("kafka1") depends_on.append("schema-registry") + if self.with_kerberized_kafka: + depends_on.append("kerberized_kafka1") + if self.with_rabbitmq: depends_on.append("rabbitmq1") @@ -1280,6 +1320,8 @@ class ClickHouseInstance: user=os.getuid(), env_file=env_file, odbc_ini_path=odbc_ini_path, + keytab_path=self.keytab_path, + krb5_conf=self.krb5_conf, entrypoint_cmd=entrypoint_cmd, networks=networks, app_net=app_net, diff --git a/tests/integration/tskk/Dockerfile b/tests/integration/tskk/Dockerfile new file mode 100644 index 0000000000..2459b6980c --- /dev/null +++ b/tests/integration/tskk/Dockerfile @@ -0,0 +1,11 @@ +FROM centos:6.6 + +# based on confluent kerberos, which is not provided as image any more + +RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation + +EXPOSE 88 749 + +ADD ./kerberos_image_config.sh /config.sh + +ENTRYPOINT ["/bin/bash", "/config.sh"] diff --git a/tests/integration/tskk/__init__.py b/tests/integration/tskk/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto b/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto new file mode 100755 index 0000000000..96b24be493 --- /dev/null +++ b/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message KeyValuePair { + uint64 key = 1; + string value = 2; +} \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format b/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format new file mode 100644 index 0000000000..c910aa8589 --- /dev/null +++ b/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format @@ -0,0 +1 @@ +(id = ${id:Escaped}, blockNo = ${blockNo:Escaped}, val1 = ${val1:CSV}, val2 = ${val2:Escaped}, val3 = ${val3:Escaped}) \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc b/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc new file mode 100644 index 0000000000..caf693313d --- /dev/null +++ b/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc @@ -0,0 +1,11 @@ +{ + "type": "record", + "name": "row", + "fields": [ + {"name": "id", "type": "long"}, + {"name": "blockNo", "type": "int"}, + {"name": "val1", "type": "string"}, + {"name": "val2", "type": "float"}, + {"name": "val3", "type": "int"} + ] + } \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp b/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp new file mode 100644 index 0000000000..44f1961205 --- /dev/null +++ b/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp @@ -0,0 +1,10 @@ +@0x99f75f775fe63dae; + +struct TestRecordStruct +{ + id @0 : Int64; + blockNo @1 : UInt16; + val1 @2 : Text; + val2 @3 : Float32; + val3 @4 : UInt8; +} \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.proto b/tests/integration/tskk/clickhouse_path/format_schemas/test.proto new file mode 100644 index 0000000000..cabdff04a7 --- /dev/null +++ b/tests/integration/tskk/clickhouse_path/format_schemas/test.proto @@ -0,0 +1,9 @@ +syntax = "proto3"; + +message TestMessage { + int64 id = 1; + uint32 blockNo = 2; + string val1 = 3; + float val2 = 4; + uint32 val3 = 5; +}; diff --git a/tests/integration/tskk/configs/kafka.xml b/tests/integration/tskk/configs/kafka.xml new file mode 100644 index 0000000000..f44ec0782a --- /dev/null +++ b/tests/integration/tskk/configs/kafka.xml @@ -0,0 +1,26 @@ + + + earliest + + SASL_PLAINTEXT + GSSAPI + kafka + /tmp/keytab/clickhouse.keytab + kafkauser/instance@TEST.CONFLUENT.IO + security + false + + + + + 300 + + 6000 + + diff --git a/tests/integration/tskk/configs/log_conf.xml b/tests/integration/tskk/configs/log_conf.xml new file mode 100644 index 0000000000..95466269af --- /dev/null +++ b/tests/integration/tskk/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + \ No newline at end of file diff --git a/tests/integration/tskk/configs/users.xml b/tests/integration/tskk/configs/users.xml new file mode 100644 index 0000000000..246e6b069e --- /dev/null +++ b/tests/integration/tskk/configs/users.xml @@ -0,0 +1,25 @@ + + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/tskk/kafka_pb2.py b/tests/integration/tskk/kafka_pb2.py new file mode 100644 index 0000000000..7989068212 --- /dev/null +++ b/tests/integration/tskk/kafka_pb2.py @@ -0,0 +1,76 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: clickhouse_path/format_schemas/kafka.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf import descriptor_pb2 +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='clickhouse_path/format_schemas/kafka.proto', + package='', + syntax='proto3', + serialized_pb=_b('\n*clickhouse_path/format_schemas/kafka.proto\"*\n\x0cKeyValuePair\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') +) +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + + + + +_KEYVALUEPAIR = _descriptor.Descriptor( + name='KeyValuePair', + full_name='KeyValuePair', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='KeyValuePair.key', index=0, + number=1, type=4, cpp_type=4, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='KeyValuePair.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=46, + serialized_end=88, +) + +DESCRIPTOR.message_types_by_name['KeyValuePair'] = _KEYVALUEPAIR + +KeyValuePair = _reflection.GeneratedProtocolMessageType('KeyValuePair', (_message.Message,), dict( + DESCRIPTOR = _KEYVALUEPAIR, + __module__ = 'clickhouse_path.format_schemas.kafka_pb2' + # @@protoc_insertion_point(class_scope:KeyValuePair) + )) +_sym_db.RegisterMessage(KeyValuePair) + + +# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/tskk/kerberos_image_config.sh b/tests/integration/tskk/kerberos_image_config.sh new file mode 100644 index 0000000000..7b7d61312b --- /dev/null +++ b/tests/integration/tskk/kerberos_image_config.sh @@ -0,0 +1,137 @@ +#!/bin/bash + +# based on confluent kerberos, which is not provided as image any more + +[[ "TRACE" ]] && set -x + +: ${REALM:=TEST.CONFLUENT.IO} +: ${DOMAIN_REALM:=test.confluent.io} +: ${KERB_MASTER_KEY:=masterkey} +: ${KERB_ADMIN_USER:=admin} +: ${KERB_ADMIN_PASS:=admin} + +create_config() { + : ${KDC_ADDRESS:=$(hostname -f)} + + cat>/etc/krb5.conf</var/kerberos/krb5kdc/kdc.conf< /var/kerberos/krb5kdc/kadm5.acl +} + +create_keytabs() { + + kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO" + + kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@TEST.CONFLUENT.IO" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@TEST.CONFLUENT.IO" + # kadmin.local -q "addprinc -randkey kafka/localhost@TEST.CONFLUENT.IO" + # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/localhost@TEST.CONFLUENT.IO" + + kadmin.local -q "addprinc -randkey zkclient@TEST.CONFLUENT.IO" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@TEST.CONFLUENT.IO" + + # kadmin.local -q "addprinc -randkey kafkauser@TEST.CONFLUENT.IO" + # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser@TEST.CONFLUENT.IO" + + kadmin.local -q "addprinc -randkey kafkauser/instance@TEST.CONFLUENT.IO" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@TEST.CONFLUENT.IO" + + chmod g+r /tmp/keytab/clickhouse.keytab + +} + +main() { + + if [ ! -f /kerberos_initialized ]; then + create_config + create_db + create_admin_user + start_kdc + + touch /kerberos_initialized + fi + + if [ ! -f /var/kerberos/krb5kdc/principal ]; then + while true; do sleep 1000; done + else + start_kdc + create_keytabs + tail -F /var/log/kerberos/krb5kdc.log + fi + +} + +[[ "$0" == "$BASH_SOURCE" ]] && main "$@" diff --git a/tests/integration/tskk/secrets/broker_jaas.conf b/tests/integration/tskk/secrets/broker_jaas.conf new file mode 100644 index 0000000000..db119698d6 --- /dev/null +++ b/tests/integration/tskk/secrets/broker_jaas.conf @@ -0,0 +1,15 @@ +KafkaServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/kerberized_kafka.keytab" + principal="kafka/kerberized_kafka1@TEST.CONFLUENT.IO"; +}; +Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/zkclient.keytab" + principal="zkclient@TEST.CONFLUENT.IO"; +}; + diff --git a/tests/integration/tskk/secrets/krb.conf b/tests/integration/tskk/secrets/krb.conf new file mode 100644 index 0000000000..fcc9ba2299 --- /dev/null +++ b/tests/integration/tskk/secrets/krb.conf @@ -0,0 +1,22 @@ +[logging] + default = FILE:/var/log/kerberos/krb5libs.log + kdc = FILE:/var/log/kerberos/krb5kdc.log + admin_server = FILE:/var/log/kerberos/kadmind.log + +[libdefaults] + default_realm = TEST.CONFLUENT.IO + dns_lookup_realm = false + dns_lookup_kdc = false + ticket_lifetime = 24h + renew_lifetime = 7d + forwardable = true + +[realms] + TEST.CONFLUENT.IO = { + kdc = kafka_kerberos + admin_server = kafka_kerberos + } + +[domain_realm] + .TEST.CONFLUENT.IO = TEST.CONFLUENT.IO + TEST.CONFLUENT.IO = TEST.CONFLUENT.IO diff --git a/tests/integration/tskk/secrets/zookeeper_jaas.conf b/tests/integration/tskk/secrets/zookeeper_jaas.conf new file mode 100644 index 0000000000..c9022d902d --- /dev/null +++ b/tests/integration/tskk/secrets/zookeeper_jaas.conf @@ -0,0 +1,14 @@ +Server { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/kafka_kerberized_zookeeper.keytab" + principal="zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO"; +}; +Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/zkclient.keytab" + principal="zkclient@TEST.CONFLUENT.IO"; +}; diff --git a/tests/integration/tskk/test.py b/tests/integration/tskk/test.py new file mode 100644 index 0000000000..348c85c6ec --- /dev/null +++ b/tests/integration/tskk/test.py @@ -0,0 +1,137 @@ +import os.path as p +import random +import threading +import time +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException +from helpers.network import PartitionManager + +import json +import subprocess +import kafka.errors +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka.admin import NewTopic +from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment +import socket +from google.protobuf.internal.encoder import _VarintBytes + +""" +protoc --version +libprotoc 3.0.0 + +# to create kafka_pb2.py +protoc --python_out=. kafka.proto +""" +import kafka_pb2 + + +# TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. +# TODO: add test for SELECT LIMIT is working. + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', + config_dir='configs', + main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], + with_kerberized_kafka=True, + # with_zookeeper=True, + clickhouse_path_dir='clickhouse_path') +kafka_id = '' # instance.cluster.kafka_docker_id + + +# Helpers + +def check_kafka_is_available(): + + # credentials are needed + + p = subprocess.Popen(('docker', + 'exec', + '-i', + kafka_id, + '/usr/bin/kafka-broker-api-versions', + '--bootstrap-server', + 'localhost:9093'), + stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + +def wait_kafka_is_available(max_retries=50): + retries = 0 + while True: + if check_kafka_is_available(): + break + else: + retries += 1 + if retries > max_retries: + raise "Kafka is not available" + print("Waiting for Kafka to start up") + time.sleep(1) + + +def kafka_produce(topic, messages, timestamp=None): + producer = KafkaProducer(bootstrap_servers="localhost:9093") + for message in messages: + producer.send(topic=topic, value=message, timestamp_ms=timestamp) + producer.flush() + print ("Produced {} messages for topic {}".format(len(messages), topic)) + + + +# Fixtures + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + global kafka_id + cluster.start() + kafka_id = instance.cluster.kerberized_kafka_docker_id + print("kafka_id is {}".format(kafka_id)) + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') + wait_kafka_is_available() + print("kafka is available - running test") + time.sleep(60) + yield # run test + +# Tests + +@pytest.mark.timeout(1000) # wait to build containers +def test_kafka_json_as_string(kafka_cluster): + kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + + instance.query(''' + CREATE TABLE test.kafka (field String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', + kafka_topic_list = 'kafka_json_as_string', + kafka_group_name = 'kafka_json_as_string', + kafka_format = 'JSONAsString', + kafka_flush_interval_ms=1000; + ''') + + result = instance.query('SELECT * FROM test.kafka;') + expected = '''\ +{"t": 123, "e": {"x": "woof"} } +{"t": 124, "e": {"x": "test"} } +{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"} +''' + assert TSV(result) == TSV(expected) + assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") + + +if __name__ == '__main__': + cluster.start() + raw_input("Cluster created, press any key to destroy...") + cluster.shutdown() diff --git a/tests/integration/tskk/test_kafka_json.reference b/tests/integration/tskk/test_kafka_json.reference new file mode 100644 index 0000000000..959bb2aad7 --- /dev/null +++ b/tests/integration/tskk/test_kafka_json.reference @@ -0,0 +1,50 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +21 21 +22 22 +23 23 +24 24 +25 25 +26 26 +27 27 +28 28 +29 29 +30 30 +31 31 +32 32 +33 33 +34 34 +35 35 +36 36 +37 37 +38 38 +39 39 +40 40 +41 41 +42 42 +43 43 +44 44 +45 45 +46 46 +47 47 +48 48 +49 49 diff --git a/tests/integration/tskk/test_kafka_virtual1.reference b/tests/integration/tskk/test_kafka_virtual1.reference new file mode 100644 index 0000000000..30c2f6e1c5 --- /dev/null +++ b/tests/integration/tskk/test_kafka_virtual1.reference @@ -0,0 +1,50 @@ + 0 virt1 0 0 0 0000-00-00 00:00:00 + 1 virt1 1 0 0 0000-00-00 00:00:00 + 2 virt1 2 0 0 0000-00-00 00:00:00 + 3 virt1 3 0 0 0000-00-00 00:00:00 + 4 virt1 4 0 0 0000-00-00 00:00:00 + 5 virt1 5 0 0 0000-00-00 00:00:00 + 6 virt1 6 0 0 0000-00-00 00:00:00 + 7 virt1 7 0 0 0000-00-00 00:00:00 + 8 virt1 8 0 0 0000-00-00 00:00:00 + 9 virt1 9 0 0 0000-00-00 00:00:00 + 10 virt1 10 0 0 0000-00-00 00:00:00 + 11 virt1 11 0 0 0000-00-00 00:00:00 + 12 virt1 12 0 0 0000-00-00 00:00:00 + 13 virt1 13 0 0 0000-00-00 00:00:00 + 14 virt1 14 0 0 0000-00-00 00:00:00 + 15 virt1 15 0 0 0000-00-00 00:00:00 + 16 virt1 16 0 0 0000-00-00 00:00:00 + 17 virt1 17 0 0 0000-00-00 00:00:00 + 18 virt1 18 0 0 0000-00-00 00:00:00 + 19 virt1 19 0 0 0000-00-00 00:00:00 + 20 virt1 20 0 0 0000-00-00 00:00:00 + 21 virt1 21 0 0 0000-00-00 00:00:00 + 22 virt1 22 0 0 0000-00-00 00:00:00 + 23 virt1 23 0 0 0000-00-00 00:00:00 + 24 virt1 24 0 0 0000-00-00 00:00:00 + 25 virt1 25 1 0 0000-00-00 00:00:00 + 26 virt1 26 1 0 0000-00-00 00:00:00 + 27 virt1 27 1 0 0000-00-00 00:00:00 + 28 virt1 28 1 0 0000-00-00 00:00:00 + 29 virt1 29 1 0 0000-00-00 00:00:00 + 30 virt1 30 1 0 0000-00-00 00:00:00 + 31 virt1 31 1 0 0000-00-00 00:00:00 + 32 virt1 32 1 0 0000-00-00 00:00:00 + 33 virt1 33 1 0 0000-00-00 00:00:00 + 34 virt1 34 1 0 0000-00-00 00:00:00 + 35 virt1 35 1 0 0000-00-00 00:00:00 + 36 virt1 36 1 0 0000-00-00 00:00:00 + 37 virt1 37 1 0 0000-00-00 00:00:00 + 38 virt1 38 1 0 0000-00-00 00:00:00 + 39 virt1 39 1 0 0000-00-00 00:00:00 + 40 virt1 40 1 0 0000-00-00 00:00:00 + 41 virt1 41 1 0 0000-00-00 00:00:00 + 42 virt1 42 1 0 0000-00-00 00:00:00 + 43 virt1 43 1 0 0000-00-00 00:00:00 + 44 virt1 44 1 0 0000-00-00 00:00:00 + 45 virt1 45 1 0 0000-00-00 00:00:00 + 46 virt1 46 1 0 0000-00-00 00:00:00 + 47 virt1 47 1 0 0000-00-00 00:00:00 + 48 virt1 48 1 0 0000-00-00 00:00:00 + 49 virt1 49 1 0 0000-00-00 00:00:00 diff --git a/tests/integration/tskk/test_kafka_virtual2.reference b/tests/integration/tskk/test_kafka_virtual2.reference new file mode 100644 index 0000000000..afb9a64f4f --- /dev/null +++ b/tests/integration/tskk/test_kafka_virtual2.reference @@ -0,0 +1,50 @@ + 0 virt2 0 0 0 0000-00-00 00:00:00 + 1 virt2 1 1 0 0000-00-00 00:00:00 + 2 virt2 2 2 0 0000-00-00 00:00:00 + 3 virt2 3 3 0 0000-00-00 00:00:00 + 4 virt2 4 4 0 0000-00-00 00:00:00 + 5 virt2 5 5 0 0000-00-00 00:00:00 + 6 virt2 6 6 0 0000-00-00 00:00:00 + 7 virt2 7 7 0 0000-00-00 00:00:00 + 8 virt2 8 8 0 0000-00-00 00:00:00 + 9 virt2 9 9 0 0000-00-00 00:00:00 + 10 virt2 10 10 0 0000-00-00 00:00:00 + 11 virt2 11 11 0 0000-00-00 00:00:00 + 12 virt2 12 12 0 0000-00-00 00:00:00 + 13 virt2 13 13 0 0000-00-00 00:00:00 + 14 virt2 14 14 0 0000-00-00 00:00:00 + 15 virt2 15 15 0 0000-00-00 00:00:00 + 16 virt2 16 16 0 0000-00-00 00:00:00 + 17 virt2 17 17 0 0000-00-00 00:00:00 + 18 virt2 18 18 0 0000-00-00 00:00:00 + 19 virt2 19 19 0 0000-00-00 00:00:00 + 20 virt2 20 20 0 0000-00-00 00:00:00 + 21 virt2 21 21 0 0000-00-00 00:00:00 + 22 virt2 22 22 0 0000-00-00 00:00:00 + 23 virt2 23 23 0 0000-00-00 00:00:00 + 24 virt2 24 24 0 0000-00-00 00:00:00 + 25 virt2 25 25 0 0000-00-00 00:00:00 + 26 virt2 26 26 0 0000-00-00 00:00:00 + 27 virt2 27 27 0 0000-00-00 00:00:00 + 28 virt2 28 28 0 0000-00-00 00:00:00 + 29 virt2 29 29 0 0000-00-00 00:00:00 + 30 virt2 30 30 0 0000-00-00 00:00:00 + 31 virt2 31 31 0 0000-00-00 00:00:00 + 32 virt2 32 32 0 0000-00-00 00:00:00 + 33 virt2 33 33 0 0000-00-00 00:00:00 + 34 virt2 34 34 0 0000-00-00 00:00:00 + 35 virt2 35 35 0 0000-00-00 00:00:00 + 36 virt2 36 36 0 0000-00-00 00:00:00 + 37 virt2 37 37 0 0000-00-00 00:00:00 + 38 virt2 38 38 0 0000-00-00 00:00:00 + 39 virt2 39 39 0 0000-00-00 00:00:00 + 40 virt2 40 40 0 0000-00-00 00:00:00 + 41 virt2 41 41 0 0000-00-00 00:00:00 + 42 virt2 42 42 0 0000-00-00 00:00:00 + 43 virt2 43 43 0 0000-00-00 00:00:00 + 44 virt2 44 44 0 0000-00-00 00:00:00 + 45 virt2 45 45 0 0000-00-00 00:00:00 + 46 virt2 46 46 0 0000-00-00 00:00:00 + 47 virt2 47 47 0 0000-00-00 00:00:00 + 48 virt2 48 48 0 0000-00-00 00:00:00 + 49 virt2 49 49 0 0000-00-00 00:00:00 From 2d5c31d5883e1ebb6bfc5f5da1266f89929574c1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 25 Aug 2020 10:44:31 +0300 Subject: [PATCH 02/15] test_storage_kerberized_kafka cleanup + negative test --- .../table-engines/integrations/kafka.md | 16 ++++++ tests/integration/helpers/cluster.py | 9 ++-- .../test_storage_kerberized_kafka/Dockerfile | 13 +++++ .../__init__.py | 0 .../configs/kafka.xml | 2 +- .../configs/log_conf.xml | 0 .../configs/users.xml | 0 .../kafka_pb2.py | 0 .../kerberos_image_config.sh | 29 +++++------ .../secrets/broker_jaas.conf | 5 +- .../secrets/krb.conf | 8 +-- .../secrets/zookeeper_jaas.conf | 4 +- .../test.py | 48 ++++++++++-------- tests/integration/tskk/Dockerfile | 11 ---- .../format_schemas/kafka.proto | 6 --- .../format_schemas/template_row.format | 1 - .../clickhouse_path/format_schemas/test.avsc | 11 ---- .../clickhouse_path/format_schemas/test.capnp | 10 ---- .../clickhouse_path/format_schemas/test.proto | 9 ---- .../tskk/test_kafka_json.reference | 50 ------------------- .../tskk/test_kafka_virtual1.reference | 50 ------------------- .../tskk/test_kafka_virtual2.reference | 50 ------------------- 22 files changed, 81 insertions(+), 251 deletions(-) create mode 100644 tests/integration/test_storage_kerberized_kafka/Dockerfile rename tests/integration/{tskk => test_storage_kerberized_kafka}/__init__.py (100%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/configs/kafka.xml (90%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/configs/log_conf.xml (100%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/configs/users.xml (100%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/kafka_pb2.py (100%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/kerberos_image_config.sh (76%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/secrets/broker_jaas.conf (74%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/secrets/krb.conf (70%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/secrets/zookeeper_jaas.conf (72%) rename tests/integration/{tskk => test_storage_kerberized_kafka}/test.py (77%) delete mode 100644 tests/integration/tskk/Dockerfile delete mode 100755 tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto delete mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/template_row.format delete mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.avsc delete mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.capnp delete mode 100644 tests/integration/tskk/clickhouse_path/format_schemas/test.proto delete mode 100644 tests/integration/tskk/test_kafka_json.reference delete mode 100644 tests/integration/tskk/test_kafka_virtual1.reference delete mode 100644 tests/integration/tskk/test_kafka_virtual2.reference diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index fe9aa2ca25..d0a4bc928a 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -165,6 +165,22 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. +### Kerberos support {#kafka-kerberos-support} + +To deal with Kerberos-aware Kafka, add `security_protocol` child element with `sasl_plaintext` value. It is enough if Kerberos ticket-granting ticket is obtained and cached by OS facilities. +ClickHouse is able to maintain Kerberos credentials using a keytab file. Consider `sasl_kerberos_service_name`, `sasl_kerberos_keytab`, `sasl_kerberos_principal` and `sasl.kerberos.kinit.cmd` child elements. + +Example: + +``` xml + + + SASL_PLAINTEXT + /home/kafkauser/kafkauser.keytab + kafkauser/kafkahost@EXAMPLE.COM + +``` + ## Virtual Columns {#virtual-columns} - `_topic` — Kafka topic. diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ab038bc702..d836fb57cf 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -619,10 +619,10 @@ class ClickHouseCluster: self.wait_schema_registry_to_start(120) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: - env = os.environ.copy() - self.kerberized_kafka_instance_path = instance.path - env['KERBERIZED_KAFKA_DIR'] = self.kerberized_kafka_instance_path + '/' - subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) + env_var = {} + env_var['KERBERIZED_KAFKA_DIR'] = instance.path + '/' + _create_env_file(self.base_dir, env_var, ".env") + subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: @@ -1211,7 +1211,6 @@ class ClickHouseInstance: shutil.copy(self.zookeeper_config_path, conf_d_dir) if self.with_kerberized_kafka: - # shutil.copytree(p.abspath(p.join(self.base_dir, 'secrets')), p.abspath(p.join(self.path, 'secrets'))) secrets_dir = p.abspath(p.join(self.custom_config_dir, os.pardir, 'secrets')) distutils.dir_util.copy_tree(secrets_dir, p.abspath(p.join(self.path, 'secrets'))) diff --git a/tests/integration/test_storage_kerberized_kafka/Dockerfile b/tests/integration/test_storage_kerberized_kafka/Dockerfile new file mode 100644 index 0000000000..2497fed7dd --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/Dockerfile @@ -0,0 +1,13 @@ +FROM centos:6.6 +# old OS to make is faster and smaller + +RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation + +EXPOSE 88 749 + +RUN touch /config.sh +# should be overwritten e.g. via docker_compose volumes +# volumes: /some_path/my_kerberos_config.sh:/config.sh:ro + + +ENTRYPOINT ["/bin/bash", "/config.sh"] diff --git a/tests/integration/tskk/__init__.py b/tests/integration/test_storage_kerberized_kafka/__init__.py similarity index 100% rename from tests/integration/tskk/__init__.py rename to tests/integration/test_storage_kerberized_kafka/__init__.py diff --git a/tests/integration/tskk/configs/kafka.xml b/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml similarity index 90% rename from tests/integration/tskk/configs/kafka.xml rename to tests/integration/test_storage_kerberized_kafka/configs/kafka.xml index f44ec0782a..0302bd78e3 100644 --- a/tests/integration/tskk/configs/kafka.xml +++ b/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml @@ -12,7 +12,7 @@ GSSAPI kafka /tmp/keytab/clickhouse.keytab - kafkauser/instance@TEST.CONFLUENT.IO + kafkauser/instance@TEST.CLICKHOUSE.TECH security false diff --git a/tests/integration/tskk/configs/log_conf.xml b/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml similarity index 100% rename from tests/integration/tskk/configs/log_conf.xml rename to tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml diff --git a/tests/integration/tskk/configs/users.xml b/tests/integration/test_storage_kerberized_kafka/configs/users.xml similarity index 100% rename from tests/integration/tskk/configs/users.xml rename to tests/integration/test_storage_kerberized_kafka/configs/users.xml diff --git a/tests/integration/tskk/kafka_pb2.py b/tests/integration/test_storage_kerberized_kafka/kafka_pb2.py similarity index 100% rename from tests/integration/tskk/kafka_pb2.py rename to tests/integration/test_storage_kerberized_kafka/kafka_pb2.py diff --git a/tests/integration/tskk/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh similarity index 76% rename from tests/integration/tskk/kerberos_image_config.sh rename to tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh index 7b7d61312b..51061ecdac 100644 --- a/tests/integration/tskk/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -1,11 +1,10 @@ #!/bin/bash -# based on confluent kerberos, which is not provided as image any more [[ "TRACE" ]] && set -x -: ${REALM:=TEST.CONFLUENT.IO} -: ${DOMAIN_REALM:=test.confluent.io} +: ${REALM:=TEST.CLICKHOUSE.TECH} +: ${DOMAIN_REALM:=test.clickhouse.tech} : ${KERB_MASTER_KEY:=masterkey} : ${KERB_ADMIN_USER:=admin} : ${KERB_ADMIN_PASS:=admin} @@ -92,23 +91,19 @@ create_admin_user() { create_keytabs() { - kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO" + kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@${REALM}" - kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@TEST.CONFLUENT.IO" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@TEST.CONFLUENT.IO" - # kadmin.local -q "addprinc -randkey kafka/localhost@TEST.CONFLUENT.IO" - # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/localhost@TEST.CONFLUENT.IO" + kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@${REALM}" - kadmin.local -q "addprinc -randkey zkclient@TEST.CONFLUENT.IO" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@TEST.CONFLUENT.IO" - - # kadmin.local -q "addprinc -randkey kafkauser@TEST.CONFLUENT.IO" - # kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser@TEST.CONFLUENT.IO" + kadmin.local -q "addprinc -randkey zkclient@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@${REALM}" + + + kadmin.local -q "addprinc -randkey kafkauser/instance@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@${REALM}" - kadmin.local -q "addprinc -randkey kafkauser/instance@TEST.CONFLUENT.IO" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@TEST.CONFLUENT.IO" - chmod g+r /tmp/keytab/clickhouse.keytab } diff --git a/tests/integration/tskk/secrets/broker_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf similarity index 74% rename from tests/integration/tskk/secrets/broker_jaas.conf rename to tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf index db119698d6..8a55ec2faa 100644 --- a/tests/integration/tskk/secrets/broker_jaas.conf +++ b/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf @@ -3,13 +3,12 @@ KafkaServer { useKeyTab=true storeKey=true keyTab="/etc/kafka/secrets/kerberized_kafka.keytab" - principal="kafka/kerberized_kafka1@TEST.CONFLUENT.IO"; + principal="kafka/kerberized_kafka1@TEST.CLICKHOUSE.TECH"; }; Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true keyTab="/etc/kafka/secrets/zkclient.keytab" - principal="zkclient@TEST.CONFLUENT.IO"; + principal="zkclient@TEST.CLICKHOUSE.TECH"; }; - diff --git a/tests/integration/tskk/secrets/krb.conf b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf similarity index 70% rename from tests/integration/tskk/secrets/krb.conf rename to tests/integration/test_storage_kerberized_kafka/secrets/krb.conf index fcc9ba2299..b36fe5bbe9 100644 --- a/tests/integration/tskk/secrets/krb.conf +++ b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf @@ -4,7 +4,7 @@ admin_server = FILE:/var/log/kerberos/kadmind.log [libdefaults] - default_realm = TEST.CONFLUENT.IO + default_realm = TEST.CLICKHOUSE.TECH dns_lookup_realm = false dns_lookup_kdc = false ticket_lifetime = 24h @@ -12,11 +12,11 @@ forwardable = true [realms] - TEST.CONFLUENT.IO = { + TEST.CLICKHOUSE.TECH = { kdc = kafka_kerberos admin_server = kafka_kerberos } [domain_realm] - .TEST.CONFLUENT.IO = TEST.CONFLUENT.IO - TEST.CONFLUENT.IO = TEST.CONFLUENT.IO + .TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH + TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH diff --git a/tests/integration/tskk/secrets/zookeeper_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf similarity index 72% rename from tests/integration/tskk/secrets/zookeeper_jaas.conf rename to tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf index c9022d902d..1b1f8103f4 100644 --- a/tests/integration/tskk/secrets/zookeeper_jaas.conf +++ b/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf @@ -3,12 +3,12 @@ Server { useKeyTab=true storeKey=true keyTab="/etc/kafka/secrets/kafka_kerberized_zookeeper.keytab" - principal="zookeeper/kafka_kerberized_zookeeper@TEST.CONFLUENT.IO"; + principal="zookeeper/kafka_kerberized_zookeeper@TEST.CLICKHOUSE.TECH"; }; Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true storeKey=true keyTab="/etc/kafka/secrets/zkclient.keytab" - principal="zkclient@TEST.CONFLUENT.IO"; + principal="zkclient@TEST.CLICKHOUSE.TECH"; }; diff --git a/tests/integration/tskk/test.py b/tests/integration/test_storage_kerberized_kafka/test.py similarity index 77% rename from tests/integration/tskk/test.py rename to tests/integration/test_storage_kerberized_kafka/test.py index 348c85c6ec..a0d147cffc 100644 --- a/tests/integration/tskk/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -17,37 +17,20 @@ from kafka.admin import NewTopic from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 from kafka.protocol.group import MemberAssignment import socket -from google.protobuf.internal.encoder import _VarintBytes - -""" -protoc --version -libprotoc 3.0.0 - -# to create kafka_pb2.py -protoc --python_out=. kafka.proto -""" -import kafka_pb2 - - -# TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. -# TODO: add test for SELECT LIMIT is working. cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', config_dir='configs', main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], with_kerberized_kafka=True, - # with_zookeeper=True, - clickhouse_path_dir='clickhouse_path') + ) kafka_id = '' # instance.cluster.kafka_docker_id - # Helpers def check_kafka_is_available(): - # credentials are needed - + # plaintext p = subprocess.Popen(('docker', 'exec', '-i', @@ -102,12 +85,11 @@ def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') wait_kafka_is_available() print("kafka is available - running test") - time.sleep(60) yield # run test # Tests -@pytest.mark.timeout(1000) # wait to build containers +@pytest.mark.timeout(180) # wait to build containers def test_kafka_json_as_string(kafka_cluster): kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) @@ -130,6 +112,30 @@ def test_kafka_json_as_string(kafka_cluster): assert TSV(result) == TSV(expected) assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") +def test_kafka_json_as_string_no_kdc(kafka_cluster): + kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + + kafka_cluster.pause_container('kafka_kerberos') + + instance.query(''' + CREATE TABLE test.kafka_no_kdc (field String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', + kafka_topic_list = 'kafka_json_as_string_no_kdc', + kafka_group_name = 'kafka_json_as_string_no_kdc', + kafka_format = 'JSONAsString', + kafka_flush_interval_ms=1000; + ''') + + result = instance.query('SELECT * FROM test.kafka_no_kdc;') + expected = '' + + kafka_cluster.unpause_container('kafka_kerberos') + + + assert TSV(result) == TSV(expected) + assert instance.contains_in_log("StorageKafka (kafka_no_kdc): Nothing to commit") + if __name__ == '__main__': cluster.start() diff --git a/tests/integration/tskk/Dockerfile b/tests/integration/tskk/Dockerfile deleted file mode 100644 index 2459b6980c..0000000000 --- a/tests/integration/tskk/Dockerfile +++ /dev/null @@ -1,11 +0,0 @@ -FROM centos:6.6 - -# based on confluent kerberos, which is not provided as image any more - -RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation - -EXPOSE 88 749 - -ADD ./kerberos_image_config.sh /config.sh - -ENTRYPOINT ["/bin/bash", "/config.sh"] diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto b/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto deleted file mode 100755 index 96b24be493..0000000000 --- a/tests/integration/tskk/clickhouse_path/format_schemas/kafka.proto +++ /dev/null @@ -1,6 +0,0 @@ -syntax = "proto3"; - -message KeyValuePair { - uint64 key = 1; - string value = 2; -} \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format b/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format deleted file mode 100644 index c910aa8589..0000000000 --- a/tests/integration/tskk/clickhouse_path/format_schemas/template_row.format +++ /dev/null @@ -1 +0,0 @@ -(id = ${id:Escaped}, blockNo = ${blockNo:Escaped}, val1 = ${val1:CSV}, val2 = ${val2:Escaped}, val3 = ${val3:Escaped}) \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc b/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc deleted file mode 100644 index caf693313d..0000000000 --- a/tests/integration/tskk/clickhouse_path/format_schemas/test.avsc +++ /dev/null @@ -1,11 +0,0 @@ -{ - "type": "record", - "name": "row", - "fields": [ - {"name": "id", "type": "long"}, - {"name": "blockNo", "type": "int"}, - {"name": "val1", "type": "string"}, - {"name": "val2", "type": "float"}, - {"name": "val3", "type": "int"} - ] - } \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp b/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp deleted file mode 100644 index 44f1961205..0000000000 --- a/tests/integration/tskk/clickhouse_path/format_schemas/test.capnp +++ /dev/null @@ -1,10 +0,0 @@ -@0x99f75f775fe63dae; - -struct TestRecordStruct -{ - id @0 : Int64; - blockNo @1 : UInt16; - val1 @2 : Text; - val2 @3 : Float32; - val3 @4 : UInt8; -} \ No newline at end of file diff --git a/tests/integration/tskk/clickhouse_path/format_schemas/test.proto b/tests/integration/tskk/clickhouse_path/format_schemas/test.proto deleted file mode 100644 index cabdff04a7..0000000000 --- a/tests/integration/tskk/clickhouse_path/format_schemas/test.proto +++ /dev/null @@ -1,9 +0,0 @@ -syntax = "proto3"; - -message TestMessage { - int64 id = 1; - uint32 blockNo = 2; - string val1 = 3; - float val2 = 4; - uint32 val3 = 5; -}; diff --git a/tests/integration/tskk/test_kafka_json.reference b/tests/integration/tskk/test_kafka_json.reference deleted file mode 100644 index 959bb2aad7..0000000000 --- a/tests/integration/tskk/test_kafka_json.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/tskk/test_kafka_virtual1.reference b/tests/integration/tskk/test_kafka_virtual1.reference deleted file mode 100644 index 30c2f6e1c5..0000000000 --- a/tests/integration/tskk/test_kafka_virtual1.reference +++ /dev/null @@ -1,50 +0,0 @@ - 0 virt1 0 0 0 0000-00-00 00:00:00 - 1 virt1 1 0 0 0000-00-00 00:00:00 - 2 virt1 2 0 0 0000-00-00 00:00:00 - 3 virt1 3 0 0 0000-00-00 00:00:00 - 4 virt1 4 0 0 0000-00-00 00:00:00 - 5 virt1 5 0 0 0000-00-00 00:00:00 - 6 virt1 6 0 0 0000-00-00 00:00:00 - 7 virt1 7 0 0 0000-00-00 00:00:00 - 8 virt1 8 0 0 0000-00-00 00:00:00 - 9 virt1 9 0 0 0000-00-00 00:00:00 - 10 virt1 10 0 0 0000-00-00 00:00:00 - 11 virt1 11 0 0 0000-00-00 00:00:00 - 12 virt1 12 0 0 0000-00-00 00:00:00 - 13 virt1 13 0 0 0000-00-00 00:00:00 - 14 virt1 14 0 0 0000-00-00 00:00:00 - 15 virt1 15 0 0 0000-00-00 00:00:00 - 16 virt1 16 0 0 0000-00-00 00:00:00 - 17 virt1 17 0 0 0000-00-00 00:00:00 - 18 virt1 18 0 0 0000-00-00 00:00:00 - 19 virt1 19 0 0 0000-00-00 00:00:00 - 20 virt1 20 0 0 0000-00-00 00:00:00 - 21 virt1 21 0 0 0000-00-00 00:00:00 - 22 virt1 22 0 0 0000-00-00 00:00:00 - 23 virt1 23 0 0 0000-00-00 00:00:00 - 24 virt1 24 0 0 0000-00-00 00:00:00 - 25 virt1 25 1 0 0000-00-00 00:00:00 - 26 virt1 26 1 0 0000-00-00 00:00:00 - 27 virt1 27 1 0 0000-00-00 00:00:00 - 28 virt1 28 1 0 0000-00-00 00:00:00 - 29 virt1 29 1 0 0000-00-00 00:00:00 - 30 virt1 30 1 0 0000-00-00 00:00:00 - 31 virt1 31 1 0 0000-00-00 00:00:00 - 32 virt1 32 1 0 0000-00-00 00:00:00 - 33 virt1 33 1 0 0000-00-00 00:00:00 - 34 virt1 34 1 0 0000-00-00 00:00:00 - 35 virt1 35 1 0 0000-00-00 00:00:00 - 36 virt1 36 1 0 0000-00-00 00:00:00 - 37 virt1 37 1 0 0000-00-00 00:00:00 - 38 virt1 38 1 0 0000-00-00 00:00:00 - 39 virt1 39 1 0 0000-00-00 00:00:00 - 40 virt1 40 1 0 0000-00-00 00:00:00 - 41 virt1 41 1 0 0000-00-00 00:00:00 - 42 virt1 42 1 0 0000-00-00 00:00:00 - 43 virt1 43 1 0 0000-00-00 00:00:00 - 44 virt1 44 1 0 0000-00-00 00:00:00 - 45 virt1 45 1 0 0000-00-00 00:00:00 - 46 virt1 46 1 0 0000-00-00 00:00:00 - 47 virt1 47 1 0 0000-00-00 00:00:00 - 48 virt1 48 1 0 0000-00-00 00:00:00 - 49 virt1 49 1 0 0000-00-00 00:00:00 diff --git a/tests/integration/tskk/test_kafka_virtual2.reference b/tests/integration/tskk/test_kafka_virtual2.reference deleted file mode 100644 index afb9a64f4f..0000000000 --- a/tests/integration/tskk/test_kafka_virtual2.reference +++ /dev/null @@ -1,50 +0,0 @@ - 0 virt2 0 0 0 0000-00-00 00:00:00 - 1 virt2 1 1 0 0000-00-00 00:00:00 - 2 virt2 2 2 0 0000-00-00 00:00:00 - 3 virt2 3 3 0 0000-00-00 00:00:00 - 4 virt2 4 4 0 0000-00-00 00:00:00 - 5 virt2 5 5 0 0000-00-00 00:00:00 - 6 virt2 6 6 0 0000-00-00 00:00:00 - 7 virt2 7 7 0 0000-00-00 00:00:00 - 8 virt2 8 8 0 0000-00-00 00:00:00 - 9 virt2 9 9 0 0000-00-00 00:00:00 - 10 virt2 10 10 0 0000-00-00 00:00:00 - 11 virt2 11 11 0 0000-00-00 00:00:00 - 12 virt2 12 12 0 0000-00-00 00:00:00 - 13 virt2 13 13 0 0000-00-00 00:00:00 - 14 virt2 14 14 0 0000-00-00 00:00:00 - 15 virt2 15 15 0 0000-00-00 00:00:00 - 16 virt2 16 16 0 0000-00-00 00:00:00 - 17 virt2 17 17 0 0000-00-00 00:00:00 - 18 virt2 18 18 0 0000-00-00 00:00:00 - 19 virt2 19 19 0 0000-00-00 00:00:00 - 20 virt2 20 20 0 0000-00-00 00:00:00 - 21 virt2 21 21 0 0000-00-00 00:00:00 - 22 virt2 22 22 0 0000-00-00 00:00:00 - 23 virt2 23 23 0 0000-00-00 00:00:00 - 24 virt2 24 24 0 0000-00-00 00:00:00 - 25 virt2 25 25 0 0000-00-00 00:00:00 - 26 virt2 26 26 0 0000-00-00 00:00:00 - 27 virt2 27 27 0 0000-00-00 00:00:00 - 28 virt2 28 28 0 0000-00-00 00:00:00 - 29 virt2 29 29 0 0000-00-00 00:00:00 - 30 virt2 30 30 0 0000-00-00 00:00:00 - 31 virt2 31 31 0 0000-00-00 00:00:00 - 32 virt2 32 32 0 0000-00-00 00:00:00 - 33 virt2 33 33 0 0000-00-00 00:00:00 - 34 virt2 34 34 0 0000-00-00 00:00:00 - 35 virt2 35 35 0 0000-00-00 00:00:00 - 36 virt2 36 36 0 0000-00-00 00:00:00 - 37 virt2 37 37 0 0000-00-00 00:00:00 - 38 virt2 38 38 0 0000-00-00 00:00:00 - 39 virt2 39 39 0 0000-00-00 00:00:00 - 40 virt2 40 40 0 0000-00-00 00:00:00 - 41 virt2 41 41 0 0000-00-00 00:00:00 - 42 virt2 42 42 0 0000-00-00 00:00:00 - 43 virt2 43 43 0 0000-00-00 00:00:00 - 44 virt2 44 44 0 0000-00-00 00:00:00 - 45 virt2 45 45 0 0000-00-00 00:00:00 - 46 virt2 46 46 0 0000-00-00 00:00:00 - 47 virt2 47 47 0 0000-00-00 00:00:00 - 48 virt2 48 48 0 0000-00-00 00:00:00 - 49 virt2 49 49 0 0000-00-00 00:00:00 From 7a6f7a872a5576270e4069f46dbb865bda0900fc Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 25 Aug 2020 11:46:03 +0300 Subject: [PATCH 03/15] forgotten docker changes --- docker/test/integration/base/Dockerfile | 3 ++- .../runner/compose/docker_compose_kerberized_kafka.yml | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 53627c7820..88f05d9482 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -17,7 +17,8 @@ RUN apt-get update \ odbc-postgresql \ sqlite3 \ curl \ - tar + tar \ + krb5-user RUN rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index f06e26c76c..01f053256f 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -42,6 +42,7 @@ services: KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true" volumes: - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/etc/kafka/secrets + - /dev/urandom:/dev/random depends_on: - kafka_kerberized_zookeeper - kafka_kerberos @@ -49,8 +50,11 @@ services: - label:disable kafka_kerberos: - build: /home/ilejn/projects/ClickHouse/tests/integration/tskk + # build: ${KERBERIZED_KAFKA_DIR:-../..--../../../../tests/integration/tskk} + image: ilejn/kdc:latest hostname: kafka_kerberos volumes: - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/tmp/keytab + - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/../../kerberos_image_config.sh:/config.sh + - /dev/urandom:/dev/random ports: [88, 749] From a2e72ec37a5c80ae0b8ed481e5928f6bb6318626 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 26 Aug 2020 23:43:02 +0300 Subject: [PATCH 04/15] test via runner is Ok, switched to custom images --- .../docker_compose_kerberized_kafka.yml | 12 ++++++------ tests/integration/helpers/cluster.py | 19 ++++++++++++++++--- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 01f053256f..8ba9ea44ea 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -11,7 +11,7 @@ services: ZOOKEEPER_SERVERS: "kafka_kerberized_zookeeper:2888:3888" KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/zookeeper_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider -Dsun.security.krb5.debug=true" volumes: - - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/etc/kafka/secrets + - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets - /dev/urandom:/dev/random depends_on: - kafka_kerberos @@ -32,7 +32,7 @@ services: # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI - KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka + KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: OUTSIDE:SASL_PLAINTEXT,UNSECURED_OUTSIDE:PLAINTEXT,UNSECURED_INSIDE:PLAINTEXT, KAFKA_INTER_BROKER_LISTENER_NAME: OUTSIDE KAFKA_BROKER_ID: 1 @@ -41,7 +41,7 @@ services: KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true" volumes: - - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/etc/kafka/secrets + - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets - /dev/urandom:/dev/random depends_on: - kafka_kerberized_zookeeper @@ -50,11 +50,11 @@ services: - label:disable kafka_kerberos: - # build: ${KERBERIZED_KAFKA_DIR:-../..--../../../../tests/integration/tskk} + # build: ${KERBERIZED_KAFKA_DIR} image: ilejn/kdc:latest hostname: kafka_kerberos volumes: - - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/secrets:/tmp/keytab - - ${KERBERIZED_KAFKA_DIR:-../../../../../tests/integration/tskk/_instances/instance}/../../kerberos_image_config.sh:/config.sh + - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab + - ${KERBERIZED_KAFKA_DIR}/../../kerberos_image_config.sh:/config.sh - /dev/urandom:/dev/random ports: [88, 749] diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d836fb57cf..358fc9d823 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -45,6 +45,10 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path +def remove_files(files): + for a_file in files: + if os.path.isfile(a_file): + os.remove(a_file) def subprocess_check_call(args): # Uncomment for debugging @@ -175,7 +179,11 @@ class ClickHouseCluster: clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, +<<<<<<< HEAD hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None, +======= + hostname=None, env_variables=None, image="ilejn/yandex_clickhouse-integration-test", +>>>>>>> test via runner is Ok, switched to custom images stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, zookeeper_docker_compose_path=None, zookeeper_use_tmpfs=True, minio_certs_dir=None): """Add an instance to the cluster. @@ -621,10 +629,15 @@ class ClickHouseCluster: if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: env_var = {} env_var['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - _create_env_file(self.base_dir, env_var, ".env") - subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') + # different docker_compose versions look for .env in different places + # -- env-file too recent to rely on it + files_to_cleanup = [] + files_to_cleanup.append(_create_env_file(self.base_dir, env_var, ".env")) + files_to_cleanup.append(_create_env_file(os.getcwd(), env_var, ".env")) + subprocess_check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') + remove_files(files_to_cleanup) if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') From c227f91636ee4dc71726806ed79663578b1d0f2e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 31 Aug 2020 11:50:04 +0300 Subject: [PATCH 05/15] remove_files improved, proper docker images, tiny cleanup --- .../docker_compose_kerberized_kafka.yml | 2 +- tests/integration/helpers/cluster.py | 8 +- .../kafka_pb2.py | 76 ------------------- .../kerberos_image_config.sh | 6 +- 4 files changed, 7 insertions(+), 85 deletions(-) delete mode 100644 tests/integration/test_storage_kerberized_kafka/kafka_pb2.py diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 8ba9ea44ea..fa41e1ee6a 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -51,7 +51,7 @@ services: kafka_kerberos: # build: ${KERBERIZED_KAFKA_DIR} - image: ilejn/kdc:latest + image: arenadata/kdc:latest hostname: kafka_kerberos volumes: - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 358fc9d823..99b8aa3876 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -47,8 +47,10 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): def remove_files(files): for a_file in files: - if os.path.isfile(a_file): + try: os.remove(a_file) + except: + pass def subprocess_check_call(args): # Uncomment for debugging @@ -179,11 +181,7 @@ class ClickHouseCluster: clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, -<<<<<<< HEAD hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None, -======= - hostname=None, env_variables=None, image="ilejn/yandex_clickhouse-integration-test", ->>>>>>> test via runner is Ok, switched to custom images stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, zookeeper_docker_compose_path=None, zookeeper_use_tmpfs=True, minio_certs_dir=None): """Add an instance to the cluster. diff --git a/tests/integration/test_storage_kerberized_kafka/kafka_pb2.py b/tests/integration/test_storage_kerberized_kafka/kafka_pb2.py deleted file mode 100644 index 7989068212..0000000000 --- a/tests/integration/test_storage_kerberized_kafka/kafka_pb2.py +++ /dev/null @@ -1,76 +0,0 @@ -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: clickhouse_path/format_schemas/kafka.proto - -import sys -_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) -from google.protobuf import descriptor as _descriptor -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database -from google.protobuf import descriptor_pb2 -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - - - -DESCRIPTOR = _descriptor.FileDescriptor( - name='clickhouse_path/format_schemas/kafka.proto', - package='', - syntax='proto3', - serialized_pb=_b('\n*clickhouse_path/format_schemas/kafka.proto\"*\n\x0cKeyValuePair\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3') -) -_sym_db.RegisterFileDescriptor(DESCRIPTOR) - - - - -_KEYVALUEPAIR = _descriptor.Descriptor( - name='KeyValuePair', - full_name='KeyValuePair', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - _descriptor.FieldDescriptor( - name='key', full_name='KeyValuePair.key', index=0, - number=1, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - _descriptor.FieldDescriptor( - name='value', full_name='KeyValuePair.value', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - syntax='proto3', - extension_ranges=[], - oneofs=[ - ], - serialized_start=46, - serialized_end=88, -) - -DESCRIPTOR.message_types_by_name['KeyValuePair'] = _KEYVALUEPAIR - -KeyValuePair = _reflection.GeneratedProtocolMessageType('KeyValuePair', (_message.Message,), dict( - DESCRIPTOR = _KEYVALUEPAIR, - __module__ = 'clickhouse_path.format_schemas.kafka_pb2' - # @@protoc_insertion_point(class_scope:KeyValuePair) - )) -_sym_db.RegisterMessage(KeyValuePair) - - -# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh index 51061ecdac..5a9669e3db 100644 --- a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -97,14 +97,14 @@ create_keytabs() { kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@${REALM}" kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@${REALM}" - kadmin.local -q "addprinc -randkey zkclient@${REALM}" + kadmin.local -q "addprinc -randkey zkclient@${REALM}" kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@${REALM}" - kadmin.local -q "addprinc -randkey kafkauser/instance@${REALM}" + kadmin.local -q "addprinc -randkey kafkauser/instance@${REALM}" kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@${REALM}" - chmod g+r /tmp/keytab/clickhouse.keytab + chmod g+r /tmp/keytab/clickhouse.keytab } From 1808a1e36ca974575972a11f028ad29504c09f8d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 2 Sep 2020 11:39:58 +0000 Subject: [PATCH 06/15] short krb ticket lifetime, Dockerile renamed --- .../{Dockerfile => Dockerfile_kdc} | 0 .../test_storage_kerberized_kafka/kerberos_image_config.sh | 4 ++-- .../test_storage_kerberized_kafka/secrets/krb.conf | 4 ++-- tests/integration/test_storage_kerberized_kafka/test.py | 1 + 4 files changed, 5 insertions(+), 4 deletions(-) rename tests/integration/test_storage_kerberized_kafka/{Dockerfile => Dockerfile_kdc} (100%) diff --git a/tests/integration/test_storage_kerberized_kafka/Dockerfile b/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc similarity index 100% rename from tests/integration/test_storage_kerberized_kafka/Dockerfile rename to tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh index 5a9669e3db..419c77e12e 100644 --- a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -22,8 +22,8 @@ create_config() { default_realm = $REALM dns_lookup_realm = false dns_lookup_kdc = false - ticket_lifetime = 24h - renew_lifetime = 7d + ticket_lifetime = 15s + renew_lifetime = 15s forwardable = true # WARNING: We use weaker key types to simplify testing as stronger key types # require the enhanced security JCE policy file to be installed. You should diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf index b36fe5bbe9..1efdf510f2 100644 --- a/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf +++ b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf @@ -7,8 +7,8 @@ default_realm = TEST.CLICKHOUSE.TECH dns_lookup_realm = false dns_lookup_kdc = false - ticket_lifetime = 24h - renew_lifetime = 7d + ticket_lifetime = 15s + renew_lifetime = 15s forwardable = true [realms] diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index a0d147cffc..92b4da88f7 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -116,6 +116,7 @@ def test_kafka_json_as_string_no_kdc(kafka_cluster): kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) kafka_cluster.pause_container('kafka_kerberos') + time.sleep(45) # wait for ticket expiration instance.query(''' CREATE TABLE test.kafka_no_kdc (field String) From 96c2a7c77ef808505d5ca8daad05314d0cc1cc4c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 9 Sep 2020 10:01:21 +0000 Subject: [PATCH 07/15] kerberized kafka test works after merge --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/test_storage_kerberized_kafka/test.py | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 99b8aa3876..ecb9babe78 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -860,6 +860,7 @@ class ClickHouseInstance: self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None + self.kerberos_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 @@ -1222,8 +1223,7 @@ class ClickHouseInstance: shutil.copy(self.zookeeper_config_path, conf_d_dir) if self.with_kerberized_kafka: - secrets_dir = p.abspath(p.join(self.custom_config_dir, os.pardir, 'secrets')) - distutils.dir_util.copy_tree(secrets_dir, p.abspath(p.join(self.path, 'secrets'))) + distutils.dir_util.copy_tree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config dir if self.custom_config_dir: diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 92b4da88f7..6ee526d982 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -20,7 +20,6 @@ import socket cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - config_dir='configs', main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], with_kerberized_kafka=True, ) From 66a055ddde7847a8876176cfd214f295c9e07cfa Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 9 Sep 2020 10:41:25 +0000 Subject: [PATCH 08/15] switched to cyrus-sasl-2.1 release branch --- .gitmodules | 1 + contrib/cyrus-sasl | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 19f93ee827..c68498fdba 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,3 +186,4 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl + branch = cyrus-sasl-2.1 diff --git a/contrib/cyrus-sasl b/contrib/cyrus-sasl index 6054630889..9995bf9d8e 160000 --- a/contrib/cyrus-sasl +++ b/contrib/cyrus-sasl @@ -1 +1 @@ -Subproject commit 6054630889fd1cd8d0659573d69badcee1e23a00 +Subproject commit 9995bf9d8e14f58934d9313ac64f13780d6dd3c9 From 23853cc8cfb7c74b2f374e3528788eaa6504b744 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 9 Sep 2020 23:23:48 +0300 Subject: [PATCH 09/15] fix conflict resolution mistake --- tests/integration/helpers/cluster.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ecb9babe78..8b11896eb4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1225,10 +1225,6 @@ class ClickHouseInstance: if self.with_kerberized_kafka: distutils.dir_util.copy_tree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) - # Copy config dir - if self.custom_config_dir: - distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir) - # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: From bcdb2277bd6eca4692b2b61a6877d129d3f4a654 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 10 Sep 2020 10:33:27 +0300 Subject: [PATCH 10/15] distutils=>shutil --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8b11896eb4..1ddb7e220e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1223,7 +1223,7 @@ class ClickHouseInstance: shutil.copy(self.zookeeper_config_path, conf_d_dir) if self.with_kerberized_kafka: - distutils.dir_util.copy_tree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) + shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) From 75836acddcf37b87a61d24d4747aecbd1daf7575 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 16 Sep 2020 00:54:58 +0300 Subject: [PATCH 11/15] code review changes --- tests/integration/helpers/cluster.py | 20 +++------------ .../Dockerfile_kdc | 2 ++ .../configs/users.xml | 25 ------------------- .../kerberos_image_config.sh | 16 ++++++------ .../test_storage_kerberized_kafka/test.py | 3 +++ 5 files changed, 16 insertions(+), 50 deletions(-) delete mode 100644 tests/integration/test_storage_kerberized_kafka/configs/users.xml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1ddb7e220e..820b102e83 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -45,13 +45,6 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path -def remove_files(files): - for a_file in files: - try: - os.remove(a_file) - except: - pass - def subprocess_check_call(args): # Uncomment for debugging # print('run:', ' ' . join(args)) @@ -625,17 +618,10 @@ class ClickHouseCluster: self.wait_schema_registry_to_start(120) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: - env_var = {} - env_var['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - - # different docker_compose versions look for .env in different places - # -- env-file too recent to rely on it - files_to_cleanup = [] - files_to_cleanup.append(_create_env_file(self.base_dir, env_var, ".env")) - files_to_cleanup.append(_create_env_file(os.getcwd(), env_var, ".env")) - subprocess_check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + env = os.environ.copy() + env['KERBERIZED_KAFKA_DIR'] = instance.path + '/' + subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') - remove_files(files_to_cleanup) if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') diff --git a/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc b/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc index 2497fed7dd..3ead8e4fe8 100644 --- a/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc +++ b/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc @@ -1,3 +1,5 @@ +# docker build -t arenadata/kdc . + FROM centos:6.6 # old OS to make is faster and smaller diff --git a/tests/integration/test_storage_kerberized_kafka/configs/users.xml b/tests/integration/test_storage_kerberized_kafka/configs/users.xml deleted file mode 100644 index 246e6b069e..0000000000 --- a/tests/integration/test_storage_kerberized_kafka/configs/users.xml +++ /dev/null @@ -1,25 +0,0 @@ - - - - - - - - - - - - - ::/0 - - default - default - - - - - - - - diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh index 419c77e12e..dda10d47d9 100644 --- a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -1,16 +1,16 @@ #!/bin/bash -[[ "TRACE" ]] && set -x +set -x # trace -: ${REALM:=TEST.CLICKHOUSE.TECH} -: ${DOMAIN_REALM:=test.clickhouse.tech} -: ${KERB_MASTER_KEY:=masterkey} -: ${KERB_ADMIN_USER:=admin} -: ${KERB_ADMIN_PASS:=admin} +: "${REALM:=TEST.CLICKHOUSE.TECH}" +: "${DOMAIN_REALM:=test.clickhouse.tech}" +: "${KERB_MASTER_KEY:=masterkey}" +: "${KERB_ADMIN_USER:=admin}" +: "${KERB_ADMIN_PASS:=admin}" create_config() { - : ${KDC_ADDRESS:=$(hostname -f)} + : "${KDC_ADDRESS:=$(hostname -f)}" cat>/etc/krb5.conf< Date: Wed, 16 Sep 2020 22:52:04 +0300 Subject: [PATCH 12/15] forgotten empty clickhouse_path dir --- .../test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR diff --git a/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR b/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR new file mode 100644 index 0000000000..e69de29bb2 From 624a4c4d55f64b67cacbab9fed4e30bc55c2d2cb Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 17 Sep 2020 16:50:52 +0300 Subject: [PATCH 13/15] kdc Dockerfile integrated --- docker/images.json | 4 ++++ .../test/integration/kerberos_kdc/Dockerfile | 2 +- .../runner/compose/docker_compose_kerberized_kafka.yml | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) rename tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc => docker/test/integration/kerberos_kdc/Dockerfile (86%) diff --git a/docker/images.json b/docker/images.json index 8c2cb35b00..e9e91864e1 100644 --- a/docker/images.json +++ b/docker/images.json @@ -133,6 +133,10 @@ "name": "yandex/clickhouse-postgresql-java-client", "dependent": [] }, + "docker/test/integration/kerberos_kdc": { + "name": "yandex/clickhouse-kerberos-kdc", + "dependent": [] + }, "docker/test/base": { "name": "yandex/clickhouse-test-base", "dependent": [ diff --git a/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc b/docker/test/integration/kerberos_kdc/Dockerfile similarity index 86% rename from tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc rename to docker/test/integration/kerberos_kdc/Dockerfile index 3ead8e4fe8..ea231b1191 100644 --- a/tests/integration/test_storage_kerberized_kafka/Dockerfile_kdc +++ b/docker/test/integration/kerberos_kdc/Dockerfile @@ -1,4 +1,4 @@ -# docker build -t arenadata/kdc . +# docker build -t yandex/clickhouse-kerberos-kdc . FROM centos:6.6 # old OS to make is faster and smaller diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index fa41e1ee6a..34c453370e 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -51,7 +51,7 @@ services: kafka_kerberos: # build: ${KERBERIZED_KAFKA_DIR} - image: arenadata/kdc:latest + image: yandex/clickhouse-kerberos-kdc:latest hostname: kafka_kerberos volumes: - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab From 494e59e7c12e0887a32b2564463a84aea500118b Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 21 Sep 2020 19:12:31 +0300 Subject: [PATCH 14/15] DOCKER_KERBEROS_KDC_TAG --- .../runner/compose/docker_compose_kerberized_kafka.yml | 3 +-- docker/test/integration/runner/dockerd-entrypoint.sh | 1 + tests/integration/runner | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 34c453370e..3ce0000b14 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -50,8 +50,7 @@ services: - label:disable kafka_kerberos: - # build: ${KERBERIZED_KAFKA_DIR} - image: yandex/clickhouse-kerberos-kdc:latest + image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG} hostname: kafka_kerberos volumes: - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c38260279e..cbdb7317b1 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -27,6 +27,7 @@ export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index f097a42e52..f38ab0aa04 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -156,6 +156,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-integration-test": env_tags += "-e {}={}".format("DOCKER_BASE_TAG", tag) + elif image == "yandex/clickhouse-kerberos-kdc": + env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: logging.info("Unknown image {}".format(image)) From e2ee97202bf7b81cae3ab05458b6c4d8609c2617 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 22 Sep 2020 23:16:34 +0300 Subject: [PATCH 15/15] USE_LIBGSASL not required to use internal librdkafka --- cmake/find/rdkafka.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index d9f815dbcd..ac11322f40 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -14,10 +14,10 @@ if (NOT ENABLE_RDKAFKA) return() endif() -if (NOT ARCH_ARM AND USE_LIBGSASL) +if (NOT ARCH_ARM) option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) elseif(USE_INTERNAL_RDKAFKA_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM}") endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt")