You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2016/03/30 22:35:07 UTC

aurora git commit: Add ACL support for announcer.

Repository: aurora
Updated Branches:
  refs/heads/master bc4649ea1 -> 55a2422f1


Add ACL support for announcer.

Reviewed at https://reviews.apache.org/r/45042/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/55a2422f
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/55a2422f
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/55a2422f

Branch: refs/heads/master
Commit: 55a2422f16f506fe2031be55d48878a06e136af8
Parents: bc4649e
Author: Kunal Thakar <ku...@gmail.com>
Authored: Wed Mar 30 13:34:44 2016 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Wed Mar 30 13:34:44 2016 -0700

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |   2 +
 docs/operations/security.md                     |  62 +++++++++-
 examples/vagrant/announcer-auth.json            |  28 +++++
 examples/vagrant/upstart/aurora-scheduler.conf  |   3 +-
 .../executor/bin/thermos_executor_main.py       |  12 +-
 .../apache/aurora/executor/common/announcer.py  |  91 ++++++++++----
 .../executor/common/announcer_zkauth_schema.py  |  41 +++++++
 .../bin/test_thermos_executor_entry_point.py    |   1 +
 .../aurora/executor/common/test_announcer.py    | 118 ++++++++++++++++++-
 9 files changed, 328 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 3a83961..4eebc9d 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -16,6 +16,8 @@
   where the root filesystem inside of the container is read-only, as it moves PEX extraction into
   the sandbox. See [here](docs/operations/configuration.md#docker-containers)
   for more detail.
+- Support for ZooKeeper authentication in the executor announcer. See
+  [here](docs/operations/security.md#announcer-authentication) for details.
 
 ### Deprecations and removals:
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/docs/operations/security.md
----------------------------------------------------------------------
diff --git a/docs/operations/security.md b/docs/operations/security.md
index 1a3d9b7..46e0b8a 100644
--- a/docs/operations/security.md
+++ b/docs/operations/security.md
@@ -4,7 +4,9 @@ Securing your Aurora Cluster
 Aurora integrates with [Apache Shiro](http://shiro.apache.org/) to provide security
 controls for its API. In addition to providing some useful features out of the box, Shiro
 also allows Aurora cluster administrators to adapt the security system to their organization’s
-existing infrastructure.
+existing infrastructure. The announcer in the Aurora thermos executor also supports security
+controls for talking to ZooKeeper.
+
 
 - [Enabling Security](#enabling-security)
 - [Authentication](#authentication)
@@ -20,13 +22,18 @@ existing infrastructure.
 - [Implementing a Custom Realm](#implementing-a-custom-realm)
 	- [Packaging a realm module](#packaging-a-realm-module)
 - [Known Issues](#known-issues)
+- [Announcer Authentication](#announcer-authentication)
+    - [ZooKeeper authentication configuration](#zookeeper-authentication-configuration)
+    - [Executor settings](#executor-settings)
 
 # Enabling Security
 
 There are two major components of security:
 [authentication and authorization](http://en.wikipedia.org/wiki/Authentication#Authorization).  A
 cluster administrator may choose the approach used for each, and may also implement custom
-mechanisms for either.  Later sections describe the options available.
+mechanisms for either.  Later sections describe the options available. To enable authentication
+ for the announcer, see [Announcer Authentication](#announcer-authentication)
+
 
 # Authentication
 
@@ -280,3 +287,54 @@ Relevant tickets:
 * [AURORA-1293](https://issues.apache.org/jira/browse/AURORA-1291): Consider defining a JSON format in place of INI
 * [AURORA-1179](https://issues.apache.org/jira/browse/AURORA-1179): Supported hashed passwords in security.ini
 * [AURORA-1295](https://issues.apache.org/jira/browse/AURORA-1295): Support security for the ReadOnlyScheduler service
+
+# Announcer Authentication
+The Thermos executor can be configured to authenticate with ZooKeeper and include
+an [ACL](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl)
+on the nodes it creates, which will specify
+the privileges of clients to perform different actions on these nodes.  This
+feature is enabled by specifying an ACL configuration file to the executor with the
+`--announcer-zookeeper-auth-config` command line argument.
+
+When this feature is _not_ enabled, nodes created by the executor will have 'world/all' permission
+(`ZOO_OPEN_ACL_UNSAFE`).  In most production environments, operators should specify an ACL and
+limit access.
+
+## ZooKeeper Authentication Configuration
+The configuration file must be formatted as JSON with the following schema:
+
+```json
+{
+  "auth": [
+    {
+      "scheme": "<scheme>",
+      "credential": "<plain_credential>"
+    }
+  ],
+  "acl": [
+    {
+      "scheme": "<scheme>",
+      "credential": "<plain_credential>",
+      "permissions": {
+        "read": <bool>,
+        "write": <bool>,
+        "create": <bool>,
+        "delete": <bool>,
+        "admin": <bool>
+      }
+    }
+  ]
+}
+```
+
+The `scheme`
+defines the encoding of the credential field.  Note that these fields are passed directly to
+ZooKeeper (except in the case of _digest_ scheme, where the executor will hash and encode
+the credential appropriately before passing it to ZooKeeper). In addition to `acl`, a list of
+authentication credentials must be provided in `auth` to use for the connection.
+
+All properties of the `permissions` object will default to False if not provided.
+
+## Executor settings
+To enable the executor to authenticate against ZK, `--announcer-zookeeper-auth-config` should be
+set to the configuration file.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/examples/vagrant/announcer-auth.json
----------------------------------------------------------------------
diff --git a/examples/vagrant/announcer-auth.json b/examples/vagrant/announcer-auth.json
new file mode 100644
index 0000000..01a24ac
--- /dev/null
+++ b/examples/vagrant/announcer-auth.json
@@ -0,0 +1,28 @@
+{
+  "auth": [
+    {
+      "scheme": "digest",
+      "credential": "user:pass"
+    }
+  ],
+  "acl": [
+    {
+      "scheme": "digest",
+      "credential": "user:pass",
+      "permissions": {
+        "read": true,
+        "create": true,
+        "delete": true,
+        "write": true
+      }
+    },
+    {
+      "scheme": "world",
+      "credential": "anyone",
+      "permissions": {
+        "read": true,
+        "delete": true
+      }
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/examples/vagrant/upstart/aurora-scheduler.conf
----------------------------------------------------------------------
diff --git a/examples/vagrant/upstart/aurora-scheduler.conf b/examples/vagrant/upstart/aurora-scheduler.conf
index 120b89a..d61801c 100644
--- a/examples/vagrant/upstart/aurora-scheduler.conf
+++ b/examples/vagrant/upstart/aurora-scheduler.conf
@@ -40,7 +40,8 @@ exec bin/aurora-scheduler \
   -native_log_file_path=/var/db/aurora \
   -backup_dir=/var/lib/aurora/backups \
   -thermos_executor_path=$DIST_DIR/thermos_executor.pex \
-  -thermos_executor_flags="--announcer-ensemble localhost:2181" \
+  -global_container_mounts=/home/vagrant/aurora/examples/vagrant/announcer-auth.json:/home/vagrant/aurora/examples/vagrant/announcer-auth.json:ro \
+  -thermos_executor_flags="--announcer-ensemble localhost:2181 --announcer-zookeeper-auth-config /home/vagrant/aurora/examples/vagrant/announcer-auth.json" \
   -allowed_container_types=MESOS,DOCKER \
   -http_authentication_mechanism=BASIC \
   -use_beta_db_task_store=true \

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py b/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
index 6634506..c4eda41 100644
--- a/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
+++ b/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
@@ -30,7 +30,7 @@ from twitter.common.log.options import LogOptions
 
 from apache.aurora.config.schema.base import LoggerDestination, LoggerMode
 from apache.aurora.executor.aurora_executor import AuroraExecutor
-from apache.aurora.executor.common.announcer import DefaultAnnouncerCheckerProvider
+from apache.aurora.executor.common.announcer import DefaultAnnouncerCheckerProvider, make_zk_auth
 from apache.aurora.executor.common.executor_timeout import ExecutorTimeout
 from apache.aurora.executor.common.health_checker import HealthCheckerProvider
 from apache.aurora.executor.common.path_detector import MesosPathDetector
@@ -101,6 +101,13 @@ app.add_option(
 )
 
 app.add_option(
+    '--announcer-zookeeper-auth-config',
+    dest='announcer_zookeeper_auth_config',
+    type=str,
+    default=None,
+    help='Path to ZooKeeper authentication to use for announcer nodes.')
+
+app.add_option(
     '--execute-as-user',
     dest='execute_as_user',
     type=str,
@@ -195,7 +202,8 @@ def initialize(options):
       options.announcer_ensemble,
       options.announcer_serverset_path,
       options.announcer_allow_custom_serverset_path,
-      options.announcer_hostname
+      options.announcer_hostname,
+      make_zk_auth(options.announcer_zookeeper_auth_config)
     ))
 
   # Create executor stub

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/src/main/python/apache/aurora/executor/common/announcer.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/common/announcer.py b/src/main/python/apache/aurora/executor/common/announcer.py
index 79a9cfb..12d8e0a 100644
--- a/src/main/python/apache/aurora/executor/common/announcer.py
+++ b/src/main/python/apache/aurora/executor/common/announcer.py
@@ -19,6 +19,7 @@ from abc import abstractmethod
 
 from kazoo.client import KazooClient
 from kazoo.retry import KazooRetry
+from kazoo.security import make_acl, make_digest_acl_credential
 from mesos.interface import mesos_pb2
 from twitter.common import app, log
 from twitter.common.concurrent.deferred import defer
@@ -27,6 +28,7 @@ from twitter.common.metrics import LambdaGauge, Observable
 from twitter.common.quantity import Amount, Time
 from twitter.common.zookeeper.serverset import Endpoint, ServerSet
 
+from apache.aurora.executor.common.announcer_zkauth_schema import ZkAuth
 from apache.aurora.executor.common.status_checker import (
     StatusChecker,
     StatusCheckerProvider,
@@ -53,6 +55,39 @@ def make_endpoints(hostname, portmap, primary_port):
   return Endpoint(hostname, portmap.get(primary_port, 0)), additional_endpoints
 
 
+def make_zk_auth(zk_auth_config):
+  if zk_auth_config is None:
+    return None
+
+  try:
+    with open(zk_auth_config) as fp:
+      try:
+        zk_auth = ZkAuth.json_load(fp, strict=True)
+        if not zk_auth.check().ok():
+          app.error('ZK authentication config is invalid %s' % zk_auth.check().message())
+        return zk_auth
+      except (TypeError, ValueError, AttributeError) as ex:
+        app.error('Problem parsing ZK authentication config %s' % ex)
+  except IOError as ie:
+    app.error('Failed to open config file %s' % ie)
+
+
+def to_acl(access):
+  cred = access.credential().get()
+  if access.scheme().get() == 'digest':
+    cred_parts = access.credential().get().split(':')
+    if len(cred_parts) != 2:
+      app.error('Digest credential should be of the form <user>:<password>')
+    cred = make_digest_acl_credential(cred_parts[0], cred_parts[1])
+  return make_acl(access.scheme().get(),
+                  cred,
+                  read=access.permissions().read().get(),
+                  write=access.permissions().write().get(),
+                  create=access.permissions().create().get(),
+                  delete=access.permissions().delete().get(),
+                  admin=access.permissions().admin().get())
+
+
 class AnnouncerCheckerProvider(StatusCheckerProvider):
   def __init__(self, allow_custom_serverset_path=False, hostname=None, name=None):
     self.name = name
@@ -118,19 +153,29 @@ class DefaultAnnouncerCheckerProvider(AnnouncerCheckerProvider):
       max_delay=DEFAULT_RETRY_MAX_DELAY.as_(Time.SECONDS),
   )
 
-  def __init__(self, ensemble, root='/aurora', allow_custom_serverset_path=False, hostname=None):
-    self.__ensemble = ensemble
-    self.__root = root
-    super(DefaultAnnouncerCheckerProvider, self).__init__(
-          allow_custom_serverset_path, hostname)
+  def __init__(self, ensemble, root='/aurora', allow_custom_serverset_path=False,
+               hostname=None, zk_auth=None):
+    self._ensemble = ensemble
+    self._root = root
+    self._zk_auth = zk_auth
+    super(DefaultAnnouncerCheckerProvider, self).__init__(allow_custom_serverset_path, hostname)
 
   def make_zk_client(self):
-    return KazooClient(self.__ensemble, connection_retry=self.DEFAULT_RETRY_POLICY)
+    if self._zk_auth is None:
+      auth_data = None
+      default_acl = None
+    else:
+      auth_data = [(a.scheme().get(), a.credential().get()) for a in self._zk_auth.auth()]
+      default_acl = [to_acl(a) for a in self._zk_auth.acl()]
+    return KazooClient(self._ensemble,
+                       connection_retry=self.DEFAULT_RETRY_POLICY,
+                       default_acl=default_acl or None,
+                       auth_data=auth_data or None)
 
   def make_zk_path(self, assigned_task):
     config = assigned_task.task
     role, environment, name = (config.job.role, config.job.environment, config.job.name)
-    return posixpath.join(self.__root, role, environment, name)
+    return posixpath.join(self._root, role, environment, name)
 
 
 class ServerSetJoinThread(ExceptionalThread):
@@ -238,34 +283,34 @@ class AnnouncerChecker(StatusChecker):
   DEFAULT_NAME = 'announcer'
 
   def __init__(self, client, path, timeout_secs, endpoint, additional=None, shard=None, name=None):
-    self.__client = client
-    self.__connect_event = client.start_async()
-    self.__timeout_secs = timeout_secs
-    self.__announcer = Announcer(ServerSet(client, path), endpoint, additional=additional,
+    self._client = client
+    self._connect_event = client.start_async()
+    self._timeout_secs = timeout_secs
+    self._announcer = Announcer(ServerSet(client, path), endpoint, additional=additional,
         shard=shard)
-    self.__name = name or self.DEFAULT_NAME
-    self.__status = None
+    self._name = name or self.DEFAULT_NAME
+    self._status = None
     self.start_event = threading.Event()
-    self.metrics.register(LambdaGauge('disconnected_time', self.__announcer.disconnected_time))
+    self.metrics.register(LambdaGauge('disconnected_time', self._announcer.disconnected_time))
 
   @property
   def status(self):
-    return self.__status
+    return self._status
 
   def name(self):
-    return self.__name
+    return self._name
 
-  def __start(self):
-    self.__connect_event.wait(timeout=self.__timeout_secs)
-    if not self.__connect_event.is_set():
-      self.__status = StatusResult("Creating Announcer Serverset timed out.", mesos_pb2.TASK_FAILED)
+  def _start(self):
+    self._connect_event.wait(timeout=self._timeout_secs)
+    if not self._connect_event.is_set():
+      self._status = StatusResult("Creating Announcer Serverset timed out.", mesos_pb2.TASK_FAILED)
     else:
-      self.__announcer.start()
+      self._announcer.start()
 
     self.start_event.set()
 
   def start(self):
-    defer(self.__start)
+    defer(self._start)
 
   def stop(self):
-    defer(self.__announcer.stop)
+    defer(self._announcer.stop)

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/src/main/python/apache/aurora/executor/common/announcer_zkauth_schema.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/common/announcer_zkauth_schema.py b/src/main/python/apache/aurora/executor/common/announcer_zkauth_schema.py
new file mode 100644
index 0000000..c84b2dd
--- /dev/null
+++ b/src/main/python/apache/aurora/executor/common/announcer_zkauth_schema.py
@@ -0,0 +1,41 @@
+#
+# Licensed 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.
+#
+
+# checkstyle: noqa
+
+from pystachio import Boolean, Default, List, Required, String, Struct
+
+
+class Auth(Struct):
+  scheme     = Required(String)
+  credential = Required(String)
+
+
+class Permissions(Struct):
+  read    = Default(Boolean, False)
+  write   = Default(Boolean, False)
+  create  = Default(Boolean, False)
+  delete  = Default(Boolean, False)
+  admin   = Default(Boolean, False)
+
+
+class Access(Struct):
+  scheme      = Required(String)
+  credential  = Required(String)
+  permissions = Required(Permissions)
+
+
+class ZkAuth(Struct):
+  auth = Default(List(Auth), [])
+  acl  = Default(List(Access), [])

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/src/test/python/apache/aurora/executor/bin/test_thermos_executor_entry_point.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/bin/test_thermos_executor_entry_point.py b/src/test/python/apache/aurora/executor/bin/test_thermos_executor_entry_point.py
index e9f7851..38deae6 100644
--- a/src/test/python/apache/aurora/executor/bin/test_thermos_executor_entry_point.py
+++ b/src/test/python/apache/aurora/executor/bin/test_thermos_executor_entry_point.py
@@ -33,6 +33,7 @@ class ThermosExecutorMainTest(unittest.TestCase):
     mock_options = Mock()
     mock_options.execute_as_user = False
     mock_options.nosetuid = False
+    mock_options.announcer_ensemble = None
     with patch(
         'apache.aurora.executor.bin.thermos_executor_main.dump_runner_pex',
         return_value=mock_dump_runner_pex):

http://git-wip-us.apache.org/repos/asf/aurora/blob/55a2422f/src/test/python/apache/aurora/executor/common/test_announcer.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/common/test_announcer.py b/src/test/python/apache/aurora/executor/common/test_announcer.py
index 142b58d..58ca3a8 100644
--- a/src/test/python/apache/aurora/executor/common/test_announcer.py
+++ b/src/test/python/apache/aurora/executor/common/test_announcer.py
@@ -12,12 +12,16 @@
 # limitations under the License.
 #
 
+import json
 import threading
 
 import pytest
 from kazoo.client import KazooClient
 from kazoo.exceptions import KazooException
+from kazoo.security import Permissions as KazooPermissions
+from kazoo.security import ACL, Id
 from mock import MagicMock, call, create_autospec, patch
+from twitter.common.contextutil import temporary_file
 from twitter.common.quantity import Amount, Time
 from twitter.common.testing.clock import ThreadedClock
 from twitter.common.zookeeper.serverset import Endpoint, ServerSet
@@ -27,8 +31,10 @@ from apache.aurora.executor.common.announcer import (
     Announcer,
     DefaultAnnouncerCheckerProvider,
     ServerSetJoinThread,
-    make_endpoints
+    make_endpoints,
+    make_zk_auth
 )
+from apache.aurora.executor.common.announcer_zkauth_schema import Access, Auth, Permissions, ZkAuth
 
 
 def test_serverset_join_thread():
@@ -326,7 +332,7 @@ def test_announcer_provider_with_zkpath(mock_client_provider, mock_serverset_pro
 @patch('apache.aurora.executor.common.announcer.ServerSet')
 @patch('apache.aurora.executor.common.announcer.KazooClient')
 @patch('apache.aurora.executor.common.announcer.Endpoint')
-def test_announcer_provider_with_hostcmd(endpoint_mock_provider,
+def test_announcer_provider_with_hostname(endpoint_mock_provider,
                                          mock_client_provider, mock_serverset_provider):
   mock_client = create_autospec(spec=KazooClient, instance=True)
   mock_client_provider.return_value = mock_client
@@ -339,3 +345,111 @@ def test_announcer_provider_with_hostcmd(endpoint_mock_provider,
   dap.from_assigned_task(assigned_task, None)
 
   assert endpoint_mock_provider.mock_calls == [call('10.2.3.4', 12345), call('10.2.3.4', 12345)]
+
+
+def generate_zk_auth_json():
+  auth = {
+    'auth': [
+      {
+        'scheme': 'digest',
+        'credential': 'user:pass'
+      }
+    ],
+    'acl': [
+      {
+        'scheme': 'digest',
+        'credential': 'user:pass',
+        'permissions': {
+          'read': True,
+          'write': True,
+          'create': True,
+          'delete': True,
+          'admin': False
+        }
+      }
+    ]
+  }
+  return json.dumps(auth)
+
+
+def test_make_zk_auth_with_good_config():
+  with temporary_file() as fp:
+    fp.write(generate_zk_auth_json())
+    fp.flush()
+
+    zk_auth = make_zk_auth(fp.name)
+    perms = Permissions(read=True, write=True, delete=True, create=True, admin=False)
+    assert zk_auth.acl()[0] == Access(scheme='digest',
+                                      credential='user:pass',
+                                      permissions=perms)
+    assert zk_auth.auth()[0] == Auth(scheme='digest', credential='user:pass')
+
+
+def test_make_zk_auth_with_no_config():
+  auth = make_zk_auth(None)
+  assert auth is None
+
+
+def test_make_zk_auth_with_bad_config():
+  with pytest.raises(SystemExit):
+    make_zk_auth('file-not-present')
+
+  with temporary_file() as fp:
+    fp.write('Bad json')
+    fp.flush()
+
+    with pytest.raises(SystemExit):
+      make_zk_auth(fp.name)
+
+
+def test_make_zk_auth_config_validation():
+  invalid_configs = [
+    # No credential in auth
+    {'auth': [{'scheme': 's'}],
+     'acl': [{'scheme': 's', 'credential': 'c', 'permissions': {'read': True}}]},
+    # Acl is not a list
+    {'auth': [{'scheme': 's', 'credential': 'c'}],
+     'acl': {'scheme': 's', 'credential': 'c', 'permissions': {'read': True}}},
+    # No credential in acl
+    {'auth': [{'scheme': 's', 'credential': 'c'}],
+     'acl': [{'scheme': 's', 'permissions': {'read': True}}]},
+    # permissions is not an object
+    {'auth': [{'scheme': 's', 'credential': 'c'}],
+     'acl': [{'scheme': 's', 'credential': 'c', 'permissions': 'non-object'}]},
+    # permissions object has unrecognized property
+    {'auth': [{'scheme': 's', 'credential': 'c'}],
+     'acl': [{'scheme': 's', 'credential': 'c', 'permissions': {'extraprop': True}}]},
+    # non boolean property in permissions object
+    {'auth': [{'scheme': 's', 'credential': 'c'}],
+     'acl': [{'scheme': 's', 'credential': 'c', 'permissions': {'read': 'non-bool'}}]},
+  ]
+  for invalid_config in invalid_configs:
+    with temporary_file() as fp:
+      fp.write(json.dumps(invalid_config))
+      fp.flush()
+
+      with pytest.raises(SystemExit):
+        make_zk_auth(fp.name)
+
+
+@patch('apache.aurora.executor.common.announcer.ServerSet')
+@patch('apache.aurora.executor.common.announcer.KazooClient')
+def test_announcer_provider_with_acl(mock_client_provider, mock_serverset_provider):
+  mock_client = create_autospec(spec=KazooClient, instance=True)
+  mock_client_provider.return_value = mock_client
+  mock_serverset = create_autospec(spec=ServerSet, instance=True)
+  mock_serverset_provider.return_value = mock_serverset
+
+  zk_auth = ZkAuth(auth=[Auth(scheme='s', credential='ca')],
+                   acl=[Access(scheme='s', credential='cl', permissions=Permissions(create=True))])
+
+  dap = DefaultAnnouncerCheckerProvider('zookeeper.example.com', '', False, None, zk_auth)
+  job = make_job('aurora', 'prod', 'proxy', 'primary', portmap={})
+  assigned_task = make_assigned_task(job, assigned_ports={'primary': 12345})
+  dap.from_assigned_task(assigned_task, None)
+
+  mock_client_provider.assert_called_once_with('zookeeper.example.com',
+                                               connection_retry=dap.DEFAULT_RETRY_POLICY,
+                                               auth_data=[('s', 'ca')],
+                                               default_acl=[ACL(KazooPermissions.CREATE,
+                                                                Id('s', 'cl'))])