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