You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by mc...@apache.org on 2014/10/06 21:59:55 UTC

git commit: Add "aurora update list" and "aurora update status" commands.

Repository: incubator-aurora
Updated Branches:
  refs/heads/master 341f4e560 -> b6610f309


Add "aurora update list" and "aurora update status" commands.

Add support for commands to query and display active updates being
managed by the scheduler. Two commands are added:

* "aurora update list", which shows all active updates that are being processed by the server.
* "aurora update status", which shows detailed status information about an update in-progress on the server.

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


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

Branch: refs/heads/master
Commit: b6610f309a24e73d7eb7db8441cfa75bf95ad16d
Parents: 341f4e5
Author: Mark Chu-Carroll <mc...@twopensource.com>
Authored: Mon Oct 6 15:45:07 2014 -0400
Committer: Mark Chu-Carroll <mc...@twitter.com>
Committed: Mon Oct 6 15:57:01 2014 -0400

----------------------------------------------------------------------
 .../python/apache/aurora/client/cli/context.py  |   4 +
 .../python/apache/aurora/client/cli/options.py  |   8 +
 .../python/apache/aurora/client/cli/update.py   | 148 ++++++++++++
 .../apache/aurora/client/cli/test_supdate.py    | 231 ++++++++++++++++++-
 .../python/apache/aurora/client/cli/util.py     |   8 +
 5 files changed, 398 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b6610f30/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 301531f..0816ac0 100644
--- a/src/main/python/apache/aurora/client/cli/context.py
+++ b/src/main/python/apache/aurora/client/cli/context.py
@@ -15,6 +15,7 @@
 from __future__ import print_function
 
 import logging
+import time
 from collections import namedtuple
 from fnmatch import fnmatch
 
@@ -214,3 +215,6 @@ class AuroraCommandContext(Context):
     if max(active_instances) < max(instances):
       raise self.CommandError(EXIT_INVALID_PARAMETER,
           "Invalid shards parameter: %s only has %s shards" % (jobkey, max(active_instances)))
+
+  def timestamp_to_string(self, timestamp):
+    return time.ctime(timestamp)

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b6610f30/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 9a81a1d..dc76c25 100644
--- a/src/main/python/apache/aurora/client/cli/options.py
+++ b/src/main/python/apache/aurora/client/cli/options.py
@@ -240,6 +240,12 @@ JOBSPEC_ARGUMENT = CommandOption('jobspec', type=jobkeytype,
     help='Fully specified job key, in CLUSTER/ROLE/ENV/NAME format')
 
 
+JOBSPEC_OPTION = CommandOption('--job', type=AuroraJobKey.from_path,
+    metavar="CLUSTER/ROLE/ENV/NAME",
+    dest="jobspec",
+    help='Fully specified job key, in CLUSTER/ROLE/ENV/NAME format')
+
+
 JSON_READ_OPTION = CommandOption('--read-json', default=False, dest='read_json',
     action='store_true',
     help='Read job configuration in json format')
@@ -261,6 +267,8 @@ NO_BATCHING_OPTION = CommandOption('--no-batching', default=False, action='store
 ROLE_ARGUMENT = CommandOption('role', type=parse_qualified_role, metavar='CLUSTER/NAME',
     help='Rolename to retrieve information about')
 
+ROLE_OPTION = CommandOption('--role', metavar='ROLENAME', default=None,
+    help='Name of the user/role')
 
 SSH_USER_OPTION = CommandOption('--ssh-user', '-l', default=None, metavar="ssh_username",
     help='ssh as this username instead of the job\'s role')

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b6610f30/src/main/python/apache/aurora/client/cli/update.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/client/cli/update.py b/src/main/python/apache/aurora/client/cli/update.py
index 41475a7..9debc91 100644
--- a/src/main/python/apache/aurora/client/cli/update.py
+++ b/src/main/python/apache/aurora/client/cli/update.py
@@ -14,6 +14,7 @@
 
 from __future__ import print_function
 
+import json
 import textwrap
 
 from apache.aurora.client.cli import EXIT_API_ERROR, EXIT_OK, Noun, Verb
@@ -22,14 +23,20 @@ from apache.aurora.client.cli.options import (
     ALL_INSTANCES,
     BIND_OPTION,
     BROWSER_OPTION,
+    CommandOption,
     CONFIG_ARGUMENT,
     HEALTHCHECK_OPTION,
     INSTANCES_SPEC_ARGUMENT,
     JOBSPEC_ARGUMENT,
+    JOBSPEC_OPTION,
     JSON_READ_OPTION,
+    JSON_WRITE_OPTION,
+    ROLE_OPTION,
     STRICT_OPTION
 )
 
+from gen.apache.aurora.api.ttypes import JobUpdateAction, JobUpdateStatus
+
 
 class StartUpdate(Verb):
   @property
@@ -144,7 +151,146 @@ class AbortUpdate(Verb):
     return EXIT_OK
 
 
+class ListUpdates(Verb):
+  @property
+  def name(self):
+    return 'list'
+
+  def get_options(self):
+    return [
+      JOBSPEC_OPTION,
+      ROLE_OPTION,
+      CommandOption("--user", default=None, metavar="username",
+          help="The name of the user who initiated the update"),
+      CommandOption("--status", choices=JobUpdateStatus._NAMES_TO_VALUES,
+          default=None,
+          action="append", help="Set of update statuses to search for"),
+      JSON_WRITE_OPTION,
+      CommandOption("cluster", metavar="clustername",
+          help="Cluster to search for matching updates")]
+
+  def help(self):
+    return """List all jobs updates, with summary info, about active updates that match a query."""
+
+  def execute(self, context):
+    api = context.get_api(context.options.cluster)
+    response = api.query_job_updates(
+        role=context.options.role,
+        jobKey=context.options.jobspec,
+        user=context.options.user,
+        update_statuses=context.options.status)
+    context.check_and_log_response(response)
+    if context.options.write_json:
+      result = []
+      for summary in response.result.getJobUpdateSummariesResult.updateSummaries:
+        job_entry = {
+            "jobkey": str(summary.jobKey),
+            "id": summary.updateId,
+            "user": summary.user,
+            "started": summary.state.createdTimestampMs,
+            "lastModified": summary.state.lastModifiedTimestampMs,
+            "status": JobUpdateStatus._VALUES_TO_NAMES[summary.state.status]
+        }
+        result.append(job_entry)
+      context.print_out(json.dumps(result, indent=2, separators=[',', ': '], sort_keys=False))
+    else:
+      for summary in response.result.getJobUpdateSummariesResult.updateSummaries:
+        created = summary.state.createdTimestampMs
+        lastMod = summary.state.lastModifiedTimestampMs
+        context.print_out("Job: %s, Id: %s, User: %s, Status: %s" % (
+            str(summary.jobKey),
+            summary.updateId,
+            summary.user,
+            JobUpdateStatus._VALUES_TO_NAMES[summary.state.status]))
+        context.print_out("Created: %s, Last Modified %s" % (created, lastMod), indent=2)
+    return EXIT_OK
+
+
+class UpdateStatus(Verb):
+  @property
+  def name(self):
+    return 'status'
+
+  def get_options(self):
+    return [JSON_WRITE_OPTION, JOBSPEC_ARGUMENT]
+
+  def help(self):
+    return """Display detailed status information about an in-progress update."""
+
+  def _get_update_id(self, context, jobkey):
+    api = context.get_api(context.options.jobspec.cluster)
+    response = api.query_job_updates(
+        jobKey=context.options.jobspec)
+    context.check_and_log_response(response, "")
+    for summary in response.result.getJobUpdateSummariesResult.updateSummaries:
+      if summary.jobKey == jobkey:
+        return summary.updateId
+    else:
+      return None
+
+  def execute(self, context):
+    id = self._get_update_id(context, context.options.jobspec)
+    api = context.get_api(context.options.jobspec.cluster)
+    response = api.get_job_update_details(id)
+    context.check_and_log_response(response)
+    details = response.result.getJobUpdateDetailsResult.details
+    if context.options.write_json:
+      result = {}
+      # the following looks odd, but it's needed to convince the json renderer
+      # to render correctly.
+      result["updateId"] = ("%s" % details.update.summary.updateId)
+      result["job"] = str(context.options.jobspec)
+      result["started"] = details.update.summary.state.createdTimestampMs
+      result["last_updated"] = details.update.summary.state.lastModifiedTimestampMs
+      result["status"] = JobUpdateStatus._VALUES_TO_NAMES[details.update.summary.state.status]
+      result["update_events"] = []
+      update_events = details.updateEvents
+      if update_events is not None and len(update_events) > 0:
+        for event in update_events:
+          result["update_events"].append({
+              "status": JobUpdateStatus._VALUES_TO_NAMES[event.status],
+              "timestampMs": event.timestampMs})
+      result["instance_update_events"] = []
+      instance_events = details.instanceEvents
+      if instance_events is not None and len(instance_events) > 0:
+        for event in instance_events:
+          result["instance_update_events"].append({
+              "instance": event.instanceId,
+              "timestamp": event.timestampMs,
+              "action": JobUpdateAction._VALUES_TO_NAMES[event.action]
+          })
+      context.print_out(json.dumps(result, indent=2, separators=[',', ': '], sort_keys=False))
+
+    else:
+      context.print_out("Job: %s, UpdateID: %s" % (context.options.jobspec,
+          details.update.summary.updateId))
+      context.print_out("Started %s, last updated: %s" %
+          (context.timestamp_to_string(details.update.summary.state.createdTimestampMs),
+          context.timestamp_to_string(
+              details.update.summary.state.lastModifiedTimestampMs)))
+      context.print_out("Current status: %s" %
+          JobUpdateStatus._VALUES_TO_NAMES[details.update.summary.state.status])
+      update_events = details.updateEvents
+      if update_events is not None and len(update_events) > 0:
+        context.print_out("Update events:")
+        for event in update_events:
+          context.print_out("Status: %s at %s" % (
+              JobUpdateStatus._VALUES_TO_NAMES[event.status],
+              context.timestamp_to_string(event.timestampMs)
+          ), indent=2)
+      instance_events = details.instanceEvents
+      if instance_events is not None and len(instance_events) > 0:
+        context.print_out("Instance events:")
+        for event in instance_events:
+          context.print_out("Instance %s at %s: %s" % (
+            event.instanceId, context.timestamp_to_string(event.timestampMs),
+            JobUpdateAction._VALUES_TO_NAMES[event.action]
+          ), indent=2)
+    return EXIT_OK
+
+
 class Update(Noun):
+
   @property
   def name(self):
     return "beta-update"
@@ -163,3 +309,5 @@ class Update(Noun):
     self.register_verb(PauseUpdate())
     self.register_verb(ResumeUpdate())
     self.register_verb(AbortUpdate())
+    self.register_verb(ListUpdates())
+    self.register_verb(UpdateStatus())

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b6610f30/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 2782fee..6775c38 100644
--- a/src/test/python/apache/aurora/client/cli/test_supdate.py
+++ b/src/test/python/apache/aurora/client/cli/test_supdate.py
@@ -12,6 +12,7 @@
 # limitations under the License.
 #
 import contextlib
+import textwrap
 
 from mock import patch
 from twitter.common.contextutil import temporary_file
@@ -19,8 +20,25 @@ 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.client import AuroraCommandLine
 from apache.aurora.client.cli.util import AuroraClientCommandTest, FakeAuroraCommandContext
+from apache.aurora.common.aurora_job_key import AuroraJobKey
 from apache.aurora.config import AuroraConfig
 
+from gen.apache.aurora.api.ttypes import (
+    GetJobUpdateDetailsResult,
+    GetJobUpdateSummariesResult,
+    JobInstanceUpdateEvent,
+    JobUpdate,
+    JobUpdateAction,
+    JobUpdateDetails,
+    JobUpdateEvent,
+    JobUpdateState,
+    JobUpdateStatus,
+    JobUpdateSummary,
+    Response,
+    ResponseCode,
+    Result
+)
+
 
 class TestUpdateCommand(AuroraClientCommandTest):
 
@@ -103,7 +121,6 @@ class TestUpdateCommand(AuroraClientCommandTest):
           "Scheduler-driven update of job west/bozo/test/hello has been resumed."]
 
   def test_update_invalid_config(self):
-    return True
     mock_context = FakeAuroraCommandContext()
     with contextlib.nested(
         patch('apache.aurora.client.cli.update.Update.create_context', return_value=mock_context),
@@ -173,3 +190,215 @@ class TestUpdateCommand(AuroraClientCommandTest):
       assert mock_context.get_out() == []
       assert mock_context.get_err() == [
           "Error: Failed to pause scheduler-driven update; see log for details"]
+
+  @classmethod
+  def get_status_query_response(cls):
+    query_response = Response()
+    query_response.responseCode = ResponseCode.OK
+    query_response.result = Result()
+    summaries = GetJobUpdateSummariesResult()
+    query_response.result.getJobUpdateSummariesResult = summaries
+    summaries.updateSummaries = [
+        JobUpdateSummary(
+            updateId="hello",
+            jobKey=AuroraJobKey('west', 'mcc', 'test', 'hello'), user="me",
+            state=JobUpdateState(status=JobUpdateStatus.ROLLING_FORWARD,
+                createdTimestampMs=1411404927, lastModifiedTimestampMs=14114056030)),
+        JobUpdateSummary(
+            updateId="goodbye",
+            jobKey=AuroraJobKey('west', 'mch', 'prod', 'goodbye'), user="me",
+            state=JobUpdateState(status=JobUpdateStatus.ROLLING_BACK,
+                createdTimestampMs=1411300632, lastModifiedTimestampMs=14114092632)),
+        JobUpdateSummary(
+            updateId="gasp",
+            jobKey=AuroraJobKey('west', 'mcq', 'devel', 'gasp'), user="me",
+            state=JobUpdateState(status=JobUpdateStatus.ROLL_FORWARD_PAUSED,
+                createdTimestampMs=1411600891, lastModifiedTimestampMs=1411800891))]
+    return query_response
+
+  def test_list_updates_command(self):
+    mock_context = FakeAuroraCommandContext()
+    mock_context.get_api('west').query_job_updates.return_value = self.get_status_query_response()
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.update.Update.create_context', return_value=mock_context),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      cmd = AuroraCommandLine()
+      result = cmd.execute(["beta-update", "list", "west", "--user=me"])
+      assert result == EXIT_OK
+      print("============\n%s\n============" % mock_context.get_out())
+      assert mock_context.get_out_str() == textwrap.dedent("""\
+          Job: west/mcc/test/hello, Id: hello, User: me, Status: ROLLING_FORWARD
+            Created: 1411404927, Last Modified 14114056030
+          Job: west/mch/prod/goodbye, Id: goodbye, User: me, Status: ROLLING_BACK
+            Created: 1411300632, Last Modified 14114092632
+          Job: west/mcq/devel/gasp, Id: gasp, User: me, Status: ROLL_FORWARD_PAUSED
+            Created: 1411600891, Last Modified 1411800891""")
+
+  def test_list_updates_command_json(self):
+    mock_context = FakeAuroraCommandContext()
+    mock_context.get_api('west').query_job_updates.return_value = self.get_status_query_response()
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.update.Update.create_context', return_value=mock_context),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      cmd = AuroraCommandLine()
+      result = cmd.execute(["beta-update", "list", "west", "--user=me", '--write-json'])
+      assert result == EXIT_OK
+      assert mock_context.get_out_str() == textwrap.dedent("""\
+          [
+            {
+              "status": "ROLLING_FORWARD",
+              "started": 1411404927,
+              "lastModified": 14114056030,
+              "user": "me",
+              "jobkey": "west/mcc/test/hello",
+              "id": "hello"
+            },
+            {
+              "status": "ROLLING_BACK",
+              "started": 1411300632,
+              "lastModified": 14114092632,
+              "user": "me",
+              "jobkey": "west/mch/prod/goodbye",
+              "id": "goodbye"
+            },
+            {
+              "status": "ROLL_FORWARD_PAUSED",
+              "started": 1411600891,
+              "lastModified": 1411800891,
+              "user": "me",
+              "jobkey": "west/mcq/devel/gasp",
+              "id": "gasp"
+            }
+          ]""")
+
+  @classmethod
+  def get_update_details_response(cls):
+    query_response = Response()
+    query_response.responseCode = ResponseCode.OK
+    query_response.result = Result()
+    details = JobUpdateDetails()
+    query_response.result.getJobUpdateDetailsResult = GetJobUpdateDetailsResult(details=details)
+    details.update = JobUpdate()
+    details.update.summary = JobUpdateSummary(
+        jobKey=AuroraJobKey('west', 'mcc', 'test', 'hello'),
+        updateId="fake-update-identifier",
+        user="me",
+        state=JobUpdateState(status=JobUpdateStatus.ROLLING_FORWARD,
+            createdTimestampMs=1411404927, lastModifiedTimestampMs=14114056030))
+    details.updateEvents = [
+      JobUpdateEvent(status=JobUpdateStatus.ROLLING_FORWARD,
+         timestampMs=1411404927),
+      JobUpdateEvent(status=JobUpdateStatus.ROLL_FORWARD_PAUSED,
+         timestampMs=1411405000),
+      JobUpdateEvent(status=JobUpdateStatus.ROLLING_FORWARD,
+         timestampMs=1411405100)
+    ]
+    details.instanceEvents = [
+      JobInstanceUpdateEvent(
+          instanceId=1,
+          timestampMs=1411404930,
+          action=JobUpdateAction.INSTANCE_UPDATING),
+      JobInstanceUpdateEvent(
+          instanceId=2,
+          timestampMs=1411404940,
+          action=JobUpdateAction.INSTANCE_UPDATING),
+      JobInstanceUpdateEvent(
+          instanceId=1,
+          timestampMs=1411404950,
+          action=JobUpdateAction.INSTANCE_UPDATED),
+      JobInstanceUpdateEvent(
+          instanceId=2,
+          timestampMs=1411404960,
+          action=JobUpdateAction.INSTANCE_UPDATED)
+    ]
+    return query_response
+
+  def test_update_status(self):
+    mock_context = FakeAuroraCommandContext()
+    api = mock_context.get_api('west')
+    api.query_job_updates.return_value = self.get_status_query_response()
+    api.get_job_update_details.return_value = self.get_update_details_response()
+
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.update.Update.create_context', return_value=mock_context),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      cmd = AuroraCommandLine()
+      result = cmd.execute(["beta-update", "status", "west/mcc/test/hello"])
+      assert result == EXIT_OK
+      print("============\n%s\n============" % mock_context.get_out())
+      assert mock_context.get_out() == [
+          "Job: west/mcc/test/hello, UpdateID: fake-update-identifier",
+          "Started YYYY-MM-DD HH:MM:SS, last updated: YYYY-MM-DD HH:MM:SS",
+          "Current status: ROLLING_FORWARD",
+          "Update events:",
+          "  Status: ROLLING_FORWARD at YYYY-MM-DD HH:MM:SS",
+          "  Status: ROLL_FORWARD_PAUSED at YYYY-MM-DD HH:MM:SS",
+          "  Status: ROLLING_FORWARD at YYYY-MM-DD HH:MM:SS",
+          "Instance events:",
+          "  Instance 1 at YYYY-MM-DD HH:MM:SS: INSTANCE_UPDATING",
+          "  Instance 2 at YYYY-MM-DD HH:MM:SS: INSTANCE_UPDATING",
+          "  Instance 1 at YYYY-MM-DD HH:MM:SS: INSTANCE_UPDATED",
+          "  Instance 2 at YYYY-MM-DD HH:MM:SS: INSTANCE_UPDATED"]
+      mock_context.get_api("west").query_job_updates.assert_called_with(jobKey=AuroraJobKey(
+          'west', 'mcc', 'test', 'hello'))
+
+  def test_update_status_json(self):
+    mock_context = FakeAuroraCommandContext()
+    api = mock_context.get_api('west')
+    api.query_job_updates.return_value = self.get_status_query_response()
+    api.get_job_update_details.return_value = self.get_update_details_response()
+
+    with contextlib.nested(
+        patch('apache.aurora.client.cli.update.Update.create_context', return_value=mock_context),
+        patch('apache.aurora.client.factory.CLUSTERS', new=self.TEST_CLUSTERS)):
+      cmd = AuroraCommandLine()
+      result = cmd.execute(["beta-update", "status", "--write-json", "west/mcc/test/hello"])
+      assert result == EXIT_OK
+      mock_context.get_api("west").query_job_updates.assert_called_with(jobKey=AuroraJobKey(
+          'west', 'mcc', 'test', 'hello'))
+      mock_context.get_api("west").get_job_update_details.assert_called_with('hello')
+      print("============\n%s\n============" % mock_context.get_out_str())
+      assert mock_context.get_out_str() == textwrap.dedent("""\
+        {
+          "status": "ROLLING_FORWARD",
+          "last_updated": 14114056030,
+          "started": 1411404927,
+          "update_events": [
+            {
+              "status": "ROLLING_FORWARD",
+              "timestampMs": 1411404927
+            },
+            {
+              "status": "ROLL_FORWARD_PAUSED",
+              "timestampMs": 1411405000
+            },
+            {
+              "status": "ROLLING_FORWARD",
+              "timestampMs": 1411405100
+            }
+          ],
+          "job": "west/mcc/test/hello",
+          "updateId": "fake-update-identifier",
+          "instance_update_events": [
+            {
+              "action": "INSTANCE_UPDATING",
+              "instance": 1,
+              "timestamp": 1411404930
+            },
+            {
+              "action": "INSTANCE_UPDATING",
+              "instance": 2,
+              "timestamp": 1411404940
+            },
+            {
+              "action": "INSTANCE_UPDATED",
+              "instance": 1,
+              "timestamp": 1411404950
+            },
+            {
+              "action": "INSTANCE_UPDATED",
+              "instance": 2,
+              "timestamp": 1411404960
+            }
+          ]
+        }""")

http://git-wip-us.apache.org/repos/asf/incubator-aurora/blob/b6610f30/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 e1ee884..ff7eda2 100644
--- a/src/test/python/apache/aurora/client/cli/util.py
+++ b/src/test/python/apache/aurora/client/cli/util.py
@@ -80,6 +80,9 @@ class FakeAuroraCommandContext(AuroraCommandContext):
   def get_out(self):
     return self.out
 
+  def get_out_str(self):
+    return '\n'.join(self.out)
+
   def get_err(self):
     return self.err
 
@@ -89,6 +92,11 @@ class FakeAuroraCommandContext(AuroraCommandContext):
   def handle_open(self, api):
     pass
 
+  def timestamp_to_string(self, ts):
+    """To fuzz around timezone issues with timestamps, just return fixed strings."""
+    return "YYYY-MM-DD HH:MM:SS"
+
+
   def add_expected_status_query_result(self, expected_result):
     self.task_status.append(expected_result)
     # each call adds an expected query result, in order.