You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gu...@apache.org on 2015/12/04 02:47:48 UTC
kafka git commit: KAFKA-2905: System test for rolling upgrade to
enable ZooKeeper ACLs with SASL
Repository: kafka
Updated Branches:
refs/heads/trunk 13e483ade -> 80df43500
KAFKA-2905: System test for rolling upgrade to enable ZooKeeper ACLs with SASL
Author: flavio junqueira <fp...@apache.org>
Reviewers: Ismael Juma, Geoff Anderson
Closes #598 from fpj/KAFKA-2905
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/80df4350
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/80df4350
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/80df4350
Branch: refs/heads/trunk
Commit: 80df43500141d6bf710b19f342c8ab172587c128
Parents: 13e483a
Author: Flavio Junqueira <fp...@apache.org>
Authored: Thu Dec 3 17:47:44 2015 -0800
Committer: Guozhang Wang <wa...@gmail.com>
Committed: Thu Dec 3 17:47:44 2015 -0800
----------------------------------------------------------------------
tests/kafkatest/services/kafka/kafka.py | 25 +++--
.../services/kafka/templates/kafka.properties | 6 ++
.../services/kafka/templates/log4j.properties | 2 +-
.../services/security/security_config.py | 5 +-
.../security/templates/gssapi_jaas.conf | 35 +++++++
tests/kafkatest/services/zookeeper.py | 35 ++++++-
.../tests/zookeeper_security_upgrade_test.py | 103 +++++++++++++++++++
7 files changed, 199 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/kafka/kafka.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py
index b2dc260..62ccc92 100644
--- a/tests/kafkatest/services/kafka/kafka.py
+++ b/tests/kafkatest/services/kafka/kafka.py
@@ -57,7 +57,8 @@ class KafkaService(JmxMixin, Service):
}
def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT,
- sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None, jmx_attributes=[]):
+ sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None,
+ jmx_attributes=[], zk_connect_timeout=5000):
"""
:type context
:type zk: ZookeeperService
@@ -75,6 +76,18 @@ class KafkaService(JmxMixin, Service):
self.sasl_mechanism = sasl_mechanism
self.topics = topics
self.minikdc = None
+ #
+ # In a heavily loaded and not very fast machine, it is
+ # sometimes necessary to give more time for the zk client
+ # to have its session established, especially if the client
+ # is authenticating and waiting for the SaslAuthenticated
+ # in addition to the SyncConnected event.
+ #
+ # The defaut value for zookeeper.connect.timeout.ms is
+ # 2 seconds and here we increase it to 5 seconds, but
+ # it can be overriden by setting the corresponding parameter
+ # for this constructor.
+ self.zk_connect_timeout = zk_connect_timeout
self.port_mappings = {
'PLAINTEXT': Port('PLAINTEXT', 9092, False),
@@ -89,7 +102,7 @@ class KafkaService(JmxMixin, Service):
@property
def security_config(self):
- return SecurityConfig(self.security_protocol, self.interbroker_security_protocol, sasl_mechanism=self.sasl_mechanism)
+ return SecurityConfig(self.security_protocol, self.interbroker_security_protocol, zk_sasl = self.zk.zk_sasl , sasl_mechanism=self.sasl_mechanism)
def open_port(self, protocol):
self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=True)
@@ -97,19 +110,19 @@ class KafkaService(JmxMixin, Service):
def close_port(self, protocol):
self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=False)
- def start_minikdc(self):
+ def start_minikdc(self, add_principals=""):
if self.security_config.has_sasl_kerberos:
if self.minikdc is None:
- self.minikdc = MiniKdc(self.context, self.nodes)
+ self.minikdc = MiniKdc(self.context, self.nodes, extra_principals = add_principals)
self.minikdc.start()
else:
self.minikdc = None
- def start(self):
+ def start(self, add_principals=""):
self.open_port(self.security_protocol)
self.open_port(self.interbroker_security_protocol)
- self.start_minikdc()
+ self.start_minikdc(add_principals)
Service.start(self)
# Create topics if necessary
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/kafka/templates/kafka.properties
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties
index a2baac1..228d84b 100644
--- a/tests/kafkatest/services/kafka/templates/kafka.properties
+++ b/tests/kafkatest/services/kafka/templates/kafka.properties
@@ -62,6 +62,12 @@ ssl.truststore.type=JKS
sasl.mechanism={{ sasl_mechanism }}
sasl.kerberos.service.name=kafka
+{% if zk_set_acl is defined %}
+zookeeper.set.acl={{zk_set_acl}}
+{% endif %}
+
+zookeeper.connection.timeout.ms={{ zk_connect_timeout }}
+
{% if replica_lag is defined %}
replica.lag.time.max.ms={{replica_lag}}
{% endif %}
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/kafka/templates/log4j.properties
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/kafka/templates/log4j.properties b/tests/kafkatest/services/kafka/templates/log4j.properties
index fb38e1f..904e51e 100644
--- a/tests/kafkatest/services/kafka/templates/log4j.properties
+++ b/tests/kafkatest/services/kafka/templates/log4j.properties
@@ -60,7 +60,7 @@ log4j.logger.kafka.producer.async.DefaultEventHandler={{ log_level }}, kafkaAppe
log4j.logger.kafka.client.ClientUtils={{ log_level }}, kafkaAppender
log4j.logger.kafka.perf={{ log_level }}, kafkaAppender
log4j.logger.kafka.perf.ProducerPerformance$ProducerThread={{ log_level }}, kafkaAppender
-log4j.logger.org.I0Itec.zkclient.ZkClient={{ log_level }}
+log4j.logger.org.I0Itec.zkclient.ZkClient={{ log_level }}, kafkaAppender
log4j.logger.kafka={{ log_level }}, kafkaAppender
log4j.logger.kafka.network.RequestChannel$={{ log_level }}, requestAppender
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/security/security_config.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py
index 912801d..b5efba8 100644
--- a/tests/kafkatest/services/security/security_config.py
+++ b/tests/kafkatest/services/security/security_config.py
@@ -74,7 +74,7 @@ class SecurityConfig(TemplateRenderer):
ssl_stores = Keytool.generate_keystore_truststore('.')
- def __init__(self, security_protocol, interbroker_security_protocol=None, sasl_mechanism=SASL_MECHANISM_GSSAPI, template_props=""):
+ def __init__(self, security_protocol=None, interbroker_security_protocol=None, sasl_mechanism=SASL_MECHANISM_GSSAPI, zk_sasl=False, template_props=""):
"""
Initialize the security properties for the node and copy
keystore and truststore to the remote node if the transport protocol
@@ -93,8 +93,9 @@ class SecurityConfig(TemplateRenderer):
if interbroker_security_protocol is None:
interbroker_security_protocol = security_protocol
self.interbroker_security_protocol = interbroker_security_protocol
- self.has_sasl = self.is_sasl(security_protocol) or self.is_sasl(interbroker_security_protocol)
+ self.has_sasl = self.is_sasl(security_protocol) or self.is_sasl(interbroker_security_protocol) or zk_sasl
self.has_ssl = self.is_ssl(security_protocol) or self.is_ssl(interbroker_security_protocol)
+ self.zk_sasl = zk_sasl
self.properties = {
'security.protocol' : security_protocol,
'ssl.keystore.location' : SecurityConfig.KEYSTORE_PATH,
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/security/templates/gssapi_jaas.conf
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/security/templates/gssapi_jaas.conf b/tests/kafkatest/services/security/templates/gssapi_jaas.conf
index 9582056..6a629d9 100644
--- a/tests/kafkatest/services/security/templates/gssapi_jaas.conf
+++ b/tests/kafkatest/services/security/templates/gssapi_jaas.conf
@@ -26,7 +26,21 @@ KafkaServer {
useKeytab="file:/mnt/security/keytab"
principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM";
};
+{% if zk_sasl %}
+Client {
+ com.ibm.security.auth.module.Krb5LoginModule required debug=false
+ credsType=both
+ useKeytab="file:/mnt/security/keytab"
+ principal="zkclient@EXAMPLE.COM";
+};
+Server {
+ com.ibm.security.auth.module.Krb5LoginModule required debug=false
+ credsType=both
+ useKeyTab="file:/mnt/security/keytab"
+ principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
+};
+{% endif %}
{% else %}
KafkaClient {
@@ -47,5 +61,26 @@ KafkaServer {
principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM";
};
+{% if zk_sasl %}
+Client {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/mnt/security/keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="zkclient@EXAMPLE.COM";
+};
+
+Server {
+ com.sun.security.auth.module.Krb5LoginModule required
+ useKeyTab=true
+ keyTab="/mnt/security/keytab"
+ storeKey=true
+ useTicketCache=false
+ principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM";
+};
{% endif %}
+{% endif %}
+
+
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/services/zookeeper.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py
index 5b64750..fb73587 100644
--- a/tests/kafkatest/services/zookeeper.py
+++ b/tests/kafkatest/services/zookeeper.py
@@ -16,6 +16,8 @@
from ducktape.services.service import Service
+from kafkatest.services.kafka.directory import kafka_dir
+from kafkatest.services.security.security_config import SecurityConfig
from kafkatest.services.kafka.directory import kafka_dir, KAFKA_TRUNK
import subprocess
@@ -34,12 +36,29 @@ class ZookeeperService(Service):
"collect_default": False}
}
- def __init__(self, context, num_nodes):
+ def __init__(self, context, num_nodes, zk_sasl = False):
"""
:type context
"""
+ self.kafka_opts = ""
+ self.zk_sasl = zk_sasl
super(ZookeeperService, self).__init__(context, num_nodes)
+ @property
+ def security_config(self):
+ return SecurityConfig(zk_sasl=self.zk_sasl)
+
+ @property
+ def security_system_properties(self):
+ return "-Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider " \
+ "-DjaasLoginRenew=3600000 " \
+ "-Djava.security.auth.login.config=%s " \
+ "-Djava.security.krb5.conf=%s " % (self.security_config.JAAS_CONF_PATH, self.security_config.KRB5CONF_PATH)
+
+ @property
+ def zk_principals(self):
+ return " zkclient " + ' '.join(['zookeeper/' + zk_node.account.hostname for zk_node in self.nodes])
+
def start_node(self, node):
idx = self.idx(node)
self.logger.info("Starting ZK node %d on %s", idx, node.account.hostname)
@@ -47,12 +66,14 @@ class ZookeeperService(Service):
node.account.ssh("mkdir -p /mnt/zookeeper")
node.account.ssh("echo %d > /mnt/zookeeper/myid" % idx)
+ self.security_config.setup_node(node)
config_file = self.render('zookeeper.properties')
self.logger.info("zookeeper.properties:")
self.logger.info(config_file)
node.account.create_file("/mnt/zookeeper.properties", config_file)
- start_cmd = "/opt/%s/bin/zookeeper-server-start.sh " % kafka_dir(node)
+ start_cmd = "export KAFKA_OPTS=\"%s\";" % self.kafka_opts
+ start_cmd += "/opt/%s/bin/zookeeper-server-start.sh " % kafka_dir(node)
start_cmd += "/mnt/zookeeper.properties 1>> %(path)s 2>> %(path)s &" % self.logs["zk_log"]
node.account.ssh(start_cmd)
@@ -85,6 +106,14 @@ class ZookeeperService(Service):
def connect_setting(self):
return ','.join([node.account.hostname + ':2181' for node in self.nodes])
+ #
+ # This call is used to simulate a rolling upgrade to enable/disable
+ # the use of ZooKeeper ACLs.
+ #
+ def zookeeper_migration(self, node, zk_acl):
+ la_migra_cmd = "/opt/%s/bin/zookeeper-security-migration.sh --zookeeper.acl=%s --zookeeper.connect=%s" % (kafka_dir(node), zk_acl, self.connect_setting())
+ node.account.ssh(la_migra_cmd)
+
def query(self, path):
"""
Queries zookeeper for data associated with 'path' and returns all fields in the schema
@@ -102,4 +131,4 @@ class ZookeeperService(Service):
match = re.match("^({.+})$", line)
if match is not None:
result = match.groups()[0]
- return result
\ No newline at end of file
+ return result
http://git-wip-us.apache.org/repos/asf/kafka/blob/80df4350/tests/kafkatest/tests/zookeeper_security_upgrade_test.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/tests/zookeeper_security_upgrade_test.py b/tests/kafkatest/tests/zookeeper_security_upgrade_test.py
new file mode 100644
index 0000000..0714832
--- /dev/null
+++ b/tests/kafkatest/tests/zookeeper_security_upgrade_test.py
@@ -0,0 +1,103 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import matrix
+
+from kafkatest.services.zookeeper import ZookeeperService
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.services.console_consumer import ConsoleConsumer, is_int
+from kafkatest.services.security.security_config import SecurityConfig
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+
+import time
+
+
+class ZooKeeperSecurityUpgradeTest(ProduceConsumeValidateTest):
+ """Tests a rolling upgrade for zookeeper.
+ """
+
+ def __init__(self, test_context):
+ super(ZooKeeperSecurityUpgradeTest, self).__init__(test_context=test_context)
+
+ def setUp(self):
+ self.topic = "test_topic"
+ self.producer_throughput = 100
+ self.num_producers = 1
+ self.num_consumers = 1
+
+ self.zk = ZookeeperService(self.test_context, num_nodes=3)
+
+ self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, topics={self.topic: {
+ "partitions": 3,
+ "replication-factor": 3,
+ 'configs': {"min.insync.replicas": 2}}})
+
+ def create_producer_and_consumer(self):
+ self.producer = VerifiableProducer(
+ self.test_context, self.num_producers, self.kafka, self.topic,
+ throughput=self.producer_throughput)
+
+ self.consumer = ConsoleConsumer(
+ self.test_context, self.num_consumers, self.kafka, self.topic,
+ consumer_timeout_ms=60000, message_validator=is_int, new_consumer=True)
+
+ self.consumer.group_id = "group"
+
+
+ @property
+ def no_sasl(self):
+ return self.kafka.security_protocol == "PLAINTEXT" or self.kafka.security_protocol == "SSL"
+
+ def run_zk_migration(self):
+ # change zk config (auth provider + jaas login)
+ self.zk.kafka_opts = self.zk.security_system_properties
+ self.zk.zk_sasl = True
+ if(self.no_sasl):
+ self.kafka.start_minikdc(self.zk.zk_principals)
+ # restart zk
+ for node in self.zk.nodes:
+ self.zk.stop_node(node)
+ self.zk.start_node(node)
+
+ # restart broker with jaas login
+ for node in self.kafka.nodes:
+ self.kafka.stop_node(node)
+ self.kafka.start_node(node)
+
+ # run migration tool
+ for node in self.zk.nodes:
+ self.zk.zookeeper_migration(node, "secure")
+
+ # restart broker with zookeeper.set.acl=true
+ self.kafka.zk_set_acl = "true"
+ for node in self.kafka.nodes:
+ self.kafka.stop_node(node)
+ self.kafka.start_node(node)
+
+ @matrix(security_protocol=["SASL_SSL","SSL","SASL_PLAINTEXT","PLAINTEXT"])
+ def test_zk_security_upgrade(self, security_protocol):
+ self.zk.start()
+ self.kafka.security_protocol = security_protocol
+ if(self.no_sasl):
+ self.kafka.start()
+ else:
+ self.kafka.start(self.zk.zk_principals)
+
+ #Create Producer and Consumer
+ self.create_producer_and_consumer()
+
+ #Run upgrade
+ self.run_produce_consume_validate(self.run_zk_migration)
\ No newline at end of file