You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by dm...@apache.org on 2014/11/13 23:24:00 UTC

incubator-aurora git commit: Add friendly error message to the client when lock is held.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master f8040b9e0 -> c464978bf


Add friendly error message to the client when lock is held.

Bugs closed: AURORA-885

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


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

Branch: refs/heads/master
Commit: c464978bf547f5567d382ddfdfa821e4bfc0fa4c
Parents: f8040b9
Author: David McLaughlin <da...@dmclaughlin.com>
Authored: Thu Nov 13 14:23:50 2014 -0800
Committer: David McLaughlin <da...@dmclaughlin.com>
Committed: Thu Nov 13 14:23:50 2014 -0800

----------------------------------------------------------------------
 .../apache/aurora/client/api/updater_util.py    |  3 +
 .../python/apache/aurora/client/cli/__init__.py |  8 ++-
 .../python/apache/aurora/client/cli/context.py  |  7 +++
 .../python/apache/aurora/client/cli/jobs.py     |  7 ++-
 .../python/apache/aurora/client/cli/options.py  | 11 ++++
 .../apache/aurora/client/cli/test_create.py     | 37 +++++++++++-
 .../apache/aurora/client/cli/test_kill.py       | 62 +++++++++++++++++++-
 .../apache/aurora/client/cli/test_restart.py    | 44 +++++++++++++-
 .../apache/aurora/client/cli/test_supdate.py    | 37 +++++++++++-
 .../apache/aurora/client/cli/test_update.py     | 40 ++++++++++++-
 .../python/apache/aurora/client/cli/util.py     | 17 +++++-
 11 files changed, 253 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/main/python/apache/aurora/client/api/updater_util.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/api/updater_util.py b/src/main/python/apache/aurora/client/api/updater_util.py
index 2dd44e3..9d2e893 100644
--- a/src/main/python/apache/aurora/client/api/updater_util.py
+++ b/src/main/python/apache/aurora/client/api/updater_util.py
@@ -97,6 +97,9 @@ class UpdaterConfig(object):
         waitForBatchCompletion=self.wait_for_batch_completion,
         updateOnlyTheseInstances=self.instances_to_ranges(instances) if instances else None)
 
+  def __eq__(self, other):
+    return self.__dict__ == other.__dict__
+
 
 class FailureThreshold(object):
   def __init__(self, max_per_instance_failures, max_total_failures):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/main/python/apache/aurora/client/cli/__init__.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/__init__.py b/src/main/python/apache/aurora/client/cli/__init__.py
index 83416e6..5dc3c3b 100644
--- a/src/main/python/apache/aurora/client/cli/__init__.py
+++ b/src/main/python/apache/aurora/client/cli/__init__.py
@@ -106,18 +106,22 @@ class Context(object):
       self.code = code
 
   def __init__(self):
-    self.options = None
+    self._options = None
     self.logging_level = None
 
   @classmethod
   def exit(cls, code, msg):
     raise cls.CommandError(code, msg)
 
+  @property
+  def options(self):
+    return self._options
+
   def set_options(self, options):
     """Add the options object to a context.
     This is separated from the constructor to make patching tests easier.
     """
-    self.options = options
+    self._options = options
 
   def set_args(self, args):
     """Add the raw argument list to a context."""

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/main/python/apache/aurora/client/cli/context.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/context.py b/src/main/python/apache/aurora/client/cli/context.py
index a5ebbdc..df9dc24 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -61,6 +61,11 @@ def bindings_to_list(bindings):
 
 
 class AuroraCommandContext(Context):
+
+  LOCK_ERROR_MSG = """Error: job is locked by an incomplete update.
+                      run 'aurora job cancel-update' to release the lock if no update is in progress
+                   """
+
   """A context object used by Aurora commands to manage command processing state
   and common operations.
   """
@@ -142,6 +147,8 @@ class AuroraCommandContext(Context):
     if resp.responseCode != ResponseCode.OK:
       if err_msg is None:
         err_msg = resp.messageDEPRECATED
+      if resp.responseCode == ResponseCode.LOCK_ERROR:
+        self.print_err(self.LOCK_ERROR_MSG)
       self.print_err(err_msg)
     self.log_response(resp)
     if resp.responseCode != ResponseCode.OK:

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/main/python/apache/aurora/client/cli/jobs.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/jobs.py b/src/main/python/apache/aurora/client/cli/jobs.py
index 28f9475..9aae558 100644
--- a/src/main/python/apache/aurora/client/cli/jobs.py
+++ b/src/main/python/apache/aurora/client/cli/jobs.py
@@ -325,8 +325,11 @@ class AbstractKillCommand(Verb):
       for i in range(min(context.options.batch_size, len(instances_to_kill))):
         batch.append(instances_to_kill.pop())
       resp = api.kill_job(job, batch)
-      if resp.responseCode is not ResponseCode.OK or self.wait_kill_tasks(
-          context, api.scheduler_proxy, job, batch) is not EXIT_OK:
+      if resp.responseCode == ResponseCode.LOCK_ERROR:
+        # Short circuit max errors in this case, and be sure to show the lock error message.
+        context.check_and_log_response(resp)
+      elif (resp.responseCode != ResponseCode.OK
+            or self.wait_kill_tasks(context, api.scheduler_proxy, job, batch) != EXIT_OK):
         context.print_err("Kill of shards %s failed with error:" % batch)
         context.log_response(resp)
         errors += 1

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/main/python/apache/aurora/client/cli/options.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/options.py b/src/main/python/apache/aurora/client/cli/options.py
index 1f5cbb6..e844cf3 100644
--- a/src/main/python/apache/aurora/client/cli/options.py
+++ b/src/main/python/apache/aurora/client/cli/options.py
@@ -58,6 +58,17 @@ class CommandOption(object):
       displayname = self.name
     return displayname
 
+  def get_destination(self):
+    """Get the attribute name this option will be stored under internally."""
+    if self.kwargs.get('dest'):
+      return self.kwargs['dest']
+    # See the spec here: https://docs.python.org/2/library/argparse.html#dest
+    return self.name.lstrip('--').replace('-', '_')
+
+  def get_default_value(self):
+    """Get the default value if no argument for this option is supplied."""
+    return self.kwargs.get('default')
+
   def render_usage(self):
     """Create a usage string for this option"""
     if not self.name.startswith('--'):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/test_create.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_create.py b/src/test/python/apache/aurora/client/cli/test_create.py
index 968c997..26a10d4 100644
--- a/src/test/python/apache/aurora/client/cli/test_create.py
+++ b/src/test/python/apache/aurora/client/cli/test_create.py
@@ -13,11 +13,14 @@
 #
 
 import contextlib
+import unittest
 
-from mock import create_autospec, patch
+import pytest
+from mock import create_autospec, Mock, patch
 from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.cli import (
+    Context,
     EXIT_COMMAND_FAILURE,
     EXIT_INTERRUPTED,
     EXIT_INVALID_CONFIGURATION,
@@ -25,13 +28,16 @@ from apache.aurora.client.cli import (
     EXIT_UNKNOWN_ERROR
 )
 from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.jobs import CreateJobCommand
+from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.config import AuroraConfig
 
-from .util import AuroraClientCommandTest, FakeAuroraCommandContext
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, mock_verb_options
 
 from gen.apache.aurora.api.ttypes import (
     AssignedTask,
     JobKey,
+    ResponseCode,
     Result,
     ScheduledTask,
     ScheduleStatus,
@@ -45,6 +51,33 @@ class UnknownException(Exception):
   pass
 
 
+class TestCreateJobCommand(unittest.TestCase):
+
+  def test_create_with_lock(self):
+    command = CreateJobCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+    mock_options = mock_verb_options(command)
+    mock_options.jobspec = jobkey
+    mock_options.config_file = "/tmp/whatever"
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    mock_config = create_autospec(spec=AuroraConfig, spec_set=True, instance=True)
+    fake_context.get_job_config = Mock(return_value=mock_config)
+    mock_api = fake_context.get_api("test")
+
+    mock_api.create_job.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    mock_api.create_job.assert_called_once_with(mock_config)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
+
+
 class TestClientCreateCommand(AuroraClientCommandTest):
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/test_kill.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_kill.py b/src/test/python/apache/aurora/client/cli/test_kill.py
index 78f5f04..de5fcb8 100644
--- a/src/test/python/apache/aurora/client/cli/test_kill.py
+++ b/src/test/python/apache/aurora/client/cli/test_kill.py
@@ -15,19 +15,22 @@
 import contextlib
 import unittest
 
+import pytest
 from mock import create_autospec, patch
 from twitter.common.contextutil import temporary_file
 
-from apache.aurora.client.cli import EXIT_TIMEOUT
+from apache.aurora.client.cli import Context, EXIT_TIMEOUT
 from apache.aurora.client.cli.client import AuroraCommandLine
-from apache.aurora.client.cli.options import parse_instances
+from apache.aurora.client.cli.jobs import KillCommand
+from apache.aurora.client.cli.options import parse_instances, TaskInstanceKey
 from apache.aurora.common.aurora_job_key import AuroraJobKey
 
 from ..api.api_util import SchedulerThriftApiSpec
-from .util import AuroraClientCommandTest, FakeAuroraCommandContext
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, mock_verb_options
 
 from gen.apache.aurora.api.ttypes import (
     JobKey,
+    ResponseCode,
     Result,
     ScheduleStatus,
     ScheduleStatusResult,
@@ -46,6 +49,59 @@ class TestInstancesParser(unittest.TestCase):
     assert parse_instances("") is None
 
 
+class TestKillCommand(unittest.TestCase):
+
+  def test_kill_lock_error_nobatch(self):
+    """Verify that the no batch code path correctly includes the lock error message."""
+    command = KillCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [])
+    mock_options.no_batching = True
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    mock_api = fake_context.get_api('test')
+    mock_api.kill_job.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    mock_api.kill_job.assert_called_once_with(jobkey, mock_options.instance_spec.instance)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
+
+  def test_kill_lock_error_batches(self):
+    """Verify that the batch kill path short circuits and includes the lock error message."""
+    command = KillCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [1])
+    mock_options.no_batching = False
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    fake_context.add_expected_status_query_result(
+      AuroraClientCommandTest.create_status_call_result(
+        AuroraClientCommandTest.create_mock_task(1, ScheduleStatus.KILLED)))
+
+    mock_api = fake_context.get_api('test')
+    mock_api.kill_job.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    mock_api.kill_job.assert_called_once_with(jobkey, mock_options.instance_spec.instance)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
+
+
 class TestClientKillCommand(AuroraClientCommandTest):
   @classmethod
   def get_kill_job_response(cls):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/test_restart.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_restart.py b/src/test/python/apache/aurora/client/cli/test_restart.py
index a8180a3..c19b21d 100644
--- a/src/test/python/apache/aurora/client/cli/test_restart.py
+++ b/src/test/python/apache/aurora/client/cli/test_restart.py
@@ -13,17 +13,55 @@
 #
 import contextlib
 import functools
+import unittest
 
+import pytest
 from mock import create_autospec, patch
 from twitter.common.contextutil import temporary_file
 
+from apache.aurora.client.api import UpdaterConfig
 from apache.aurora.client.api.health_check import Retriable, StatusHealthCheck
-from apache.aurora.client.cli import EXIT_API_ERROR, EXIT_INVALID_PARAMETER
+from apache.aurora.client.cli import Context, EXIT_API_ERROR, EXIT_INVALID_PARAMETER
 from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.jobs import RestartCommand
+from apache.aurora.client.cli.options import TaskInstanceKey
+from apache.aurora.common.aurora_job_key import AuroraJobKey
 
-from .util import AuroraClientCommandTest, IOMock
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, IOMock, mock_verb_options
 
-from gen.apache.aurora.api.ttypes import JobKey, PopulateJobResult, Result, TaskConfig
+from gen.apache.aurora.api.ttypes import JobKey, PopulateJobResult, ResponseCode, Result, TaskConfig
+
+
+class TestRestartJobCommand(unittest.TestCase):
+
+  def test_restart_with_lock(self):
+    command = RestartCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [])
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    mock_api = fake_context.get_api("test")
+    mock_api.restart.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    updater_config = UpdaterConfig(
+      mock_options.batch_size,
+      mock_options.restart_threshold,
+      mock_options.watch_secs,
+      mock_options.max_per_instance_failures,
+      mock_options.max_total_failures,
+      mock_options.rollback_on_failure)
+
+    mock_api.restart.assert_called_once_with(jobkey, mock_options.instance_spec.instance,
+      updater_config, mock_options.healthcheck_interval_seconds, config=None)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
 
 
 class TestRestartCommand(AuroraClientCommandTest):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/test_supdate.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_supdate.py b/src/test/python/apache/aurora/client/cli/test_supdate.py
index 09f6a85..97a9664 100644
--- a/src/test/python/apache/aurora/client/cli/test_supdate.py
+++ b/src/test/python/apache/aurora/client/cli/test_supdate.py
@@ -13,16 +13,20 @@
 #
 import contextlib
 import textwrap
+import unittest
 
-from mock import patch
+import pytest
+from mock import create_autospec, Mock, patch
 from twitter.common.contextutil import temporary_file
 
-from apache.aurora.client.cli import EXIT_API_ERROR, EXIT_INVALID_CONFIGURATION, EXIT_OK
+from apache.aurora.client.cli import Context, EXIT_API_ERROR, EXIT_INVALID_CONFIGURATION, EXIT_OK
 from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.options import TaskInstanceKey
+from apache.aurora.client.cli.update import StartUpdate
 from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.config import AuroraConfig
 
-from .util import AuroraClientCommandTest, FakeAuroraCommandContext
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, mock_verb_options
 
 from gen.apache.aurora.api.ttypes import (
     GetJobUpdateDetailsResult,
@@ -43,6 +47,33 @@ from gen.apache.aurora.api.ttypes import (
 )
 
 
+class TestStartUpdateCommand(unittest.TestCase):
+
+  def test_start_update_with_lock(self):
+    command = StartUpdate()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [])
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    mock_config = create_autospec(spec=AuroraConfig, spec_set=True, instance=True)
+    fake_context.get_job_config = Mock(return_value=mock_config)
+
+    mock_api = fake_context.get_api("test")
+    mock_api.start_job_update.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    mock_api.start_job_update.assert_called_once_with(mock_config,
+      mock_options.instance_spec.instance)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
+
+
 class TestUpdateCommand(AuroraClientCommandTest):
 
   def test_start_update_command_line_succeeds(self):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/test_update.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/test_update.py b/src/test/python/apache/aurora/client/cli/test_update.py
index a5e59e4..1f061a3 100644
--- a/src/test/python/apache/aurora/client/cli/test_update.py
+++ b/src/test/python/apache/aurora/client/cli/test_update.py
@@ -13,19 +13,24 @@
 #
 import contextlib
 import functools
+import unittest
 
-from mock import create_autospec, patch
+import pytest
+from mock import create_autospec, Mock, patch
 from twitter.common.contextutil import temporary_file
 
 from apache.aurora.client.api.health_check import Retriable, StatusHealthCheck
 from apache.aurora.client.api.job_monitor import JobMonitor
 from apache.aurora.client.api.quota_check import QuotaCheck
 from apache.aurora.client.api.scheduler_mux import SchedulerMux
-from apache.aurora.client.cli import EXIT_INVALID_CONFIGURATION, EXIT_OK
+from apache.aurora.client.cli import Context, EXIT_INVALID_CONFIGURATION, EXIT_OK
 from apache.aurora.client.cli.client import AuroraCommandLine
+from apache.aurora.client.cli.jobs import UpdateCommand
+from apache.aurora.client.cli.options import TaskInstanceKey
+from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.config import AuroraConfig
 
-from .util import AuroraClientCommandTest, FakeAuroraCommandContext, IOMock
+from .util import AuroraClientCommandTest, FakeAuroraCommandContext, IOMock, mock_verb_options
 
 from gen.apache.aurora.api.constants import ACTIVE_STATES
 from gen.apache.aurora.api.ttypes import (
@@ -45,6 +50,35 @@ from gen.apache.aurora.api.ttypes import (
 )
 
 
+class TestJobUpdateCommand(unittest.TestCase):
+
+  def test_update_with_lock(self):
+    command = UpdateCommand()
+
+    jobkey = AuroraJobKey("cluster", "role", "env", "job")
+    mock_options = mock_verb_options(command)
+    mock_options.instance_spec = TaskInstanceKey(jobkey, [])
+    mock_options.force = True
+
+    fake_context = FakeAuroraCommandContext()
+    fake_context.set_options(mock_options)
+
+    mock_config = create_autospec(spec=AuroraConfig, spec_set=True, instance=True)
+    fake_context.get_job_config = Mock(return_value=mock_config)
+
+    mock_api = fake_context.get_api("test")
+    mock_api.update_job.return_value = AuroraClientCommandTest.create_blank_response(
+      ResponseCode.LOCK_ERROR, "Error.")
+
+    with pytest.raises(Context.CommandError):
+      command.execute(fake_context)
+
+    mock_api.update_job.assert_called_once_with(mock_config,
+      mock_options.healthcheck_interval_seconds,
+      mock_options.instance_spec.instance)
+    assert fake_context.get_err()[0] == fake_context.LOCK_ERROR_MSG
+
+
 class TestUpdateCommand(AuroraClientCommandTest):
   class FakeSchedulerMux(SchedulerMux):
     def enqueue_and_wait(self, command, data, aggregator=None, timeout=None):

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/c464978b/src/test/python/apache/aurora/client/cli/util.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/client/cli/util.py b/src/test/python/apache/aurora/client/cli/util.py
index 154fb3a..a89211a 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -15,7 +15,7 @@
 import textwrap
 import unittest
 
-from mock import create_autospec
+from mock import create_autospec, Mock
 
 from apache.aurora.client.cli.context import AuroraCommandContext
 from apache.aurora.client.hooks.hooked_api import HookedAuroraClientAPI
@@ -42,10 +42,23 @@ from gen.apache.aurora.api.ttypes import (
 )
 
 
+def mock_verb_options(verb):
+  return create_options_mock(verb.get_options())
+
+
+def create_options_mock(options):
+  """Inspects a list of CommandOption instances for their attribute and default values and
+     constructs a mock with specification from it."""
+  attributes = [o.get_destination() for o in options]
+  mock = Mock(spec_set=attributes)
+  for o in options:
+    setattr(mock, o.get_destination(), o.get_default_value())
+  return mock
+
+
 class FakeAuroraCommandContext(AuroraCommandContext):
   def __init__(self):
     super(FakeAuroraCommandContext, self).__init__()
-    self.options = None
     self.status = []
     self.fake_api = self.create_mock_api()
     self.task_status = []