You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by jc...@apache.org on 2016/12/08 02:41:18 UTC

aurora git commit: Extend warm-up time by `max_consecutive_failures` attempts.

Repository: aurora
Updated Branches:
  refs/heads/master 0c1770581 -> d05ea2bcc


Extend warm-up time by `max_consecutive_failures` attempts.

It is possible to set the health checks such that a task can
continually fail health checks with intermittent successes and still
succeed an update. Essentially a task fails health checks during the
`initial_interval_secs` and an additional `max_consecutive_failures`,
and then perform a successful health check to become healthy.

To be backward compatible to the above configuration, include the
`max_consecutive_failures` when computing `max_attempts_to_running`.

Bugs closed: AURORA-1841

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


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

Branch: refs/heads/master
Commit: d05ea2bcc4af20af0b5e0a7f3995d02ff25a1626
Parents: 0c17705
Author: Santhosh Kumar Shanmugham <sa...@gmail.com>
Authored: Wed Dec 7 20:40:59 2016 -0600
Committer: Joshua Cohen <jc...@apache.org>
Committed: Wed Dec 7 20:40:59 2016 -0600

----------------------------------------------------------------------
 docs/features/services.md                       |   6 +-
 .../apache/aurora/executor/aurora_executor.py   |   6 +-
 .../aurora/executor/common/health_checker.py    |  27 +-
 .../executor/common/test_health_checker.py      | 271 ++++++++++++++++---
 4 files changed, 262 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/d05ea2bc/docs/features/services.md
----------------------------------------------------------------------
diff --git a/docs/features/services.md b/docs/features/services.md
index 50189ee..f26b4cc 100644
--- a/docs/features/services.md
+++ b/docs/features/services.md
@@ -94,9 +94,9 @@ Starting with the 0.17.0 release, job updates rely only on task health-checks by
 a `min_consecutive_successes` parameter on the HealthCheckConfig object. This parameter represents
 the number of successful health checks needed before a task is moved into the `RUNNING` state. Tasks
 that do not have enough successful health checks within the first `n` attempts, are moved to the
-`FAILED` state, where `n = ceil(initial_interval_secs/interval_secs) + min_consecutive_successes`.
-In order to accommodate variability during task warm up, `initial_interval_secs` will
-act as a grace period. Any health-check failures during the first `m` attempts are ignored and
+`FAILED` state, where `n = ceil(initial_interval_secs/interval_secs) + max_consecutive_failures +
+min_consecutive_successes`. In order to accommodate variability during task warm up, `initial_interval_secs`
+will act as a grace period. Any health-check failures during the first `m` attempts are ignored and
 do not count towards `max_consecutive_failures`, where `m = ceil(initial_interval_secs/interval_secs)`.
 
 As [job updates](job-updates.md) are based only on health-checks, it is not necessary to set

http://git-wip-us.apache.org/repos/asf/aurora/blob/d05ea2bc/src/main/python/apache/aurora/executor/aurora_executor.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/aurora_executor.py b/src/main/python/apache/aurora/executor/aurora_executor.py
index d01fcb9..81461cb 100644
--- a/src/main/python/apache/aurora/executor/aurora_executor.py
+++ b/src/main/python/apache/aurora/executor/aurora_executor.py
@@ -184,9 +184,9 @@ class AuroraExecutor(ExecutorBase, Observable):
     self._status_manager.start()
     self.status_manager_started.set()
 
-  def _signal_running(self, reason):
-    log.info('Send TASK_RUNNING status update. reason: %s' % reason)
-    self.send_update(self._driver, self._task_id, mesos_pb2.TASK_RUNNING, reason)
+  def _signal_running(self, status_result):
+    log.info('Send TASK_RUNNING status update. status: %s' % status_result)
+    self.send_update(self._driver, self._task_id, mesos_pb2.TASK_RUNNING, status_result.reason)
 
   def _signal_kill_manager(self, driver, task_id, reason):
     if self._task_id is None:

http://git-wip-us.apache.org/repos/asf/aurora/blob/d05ea2bc/src/main/python/apache/aurora/executor/common/health_checker.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/common/health_checker.py b/src/main/python/apache/aurora/executor/common/health_checker.py
index 12af9d8..a5fb18f 100644
--- a/src/main/python/apache/aurora/executor/common/health_checker.py
+++ b/src/main/python/apache/aurora/executor/common/health_checker.py
@@ -102,9 +102,19 @@ class ThreadedHealthChecker(ExceptionalThread):
     # to guarantee the number of health checks during the grace period.
     # Relying on time might cause non-deterministic behavior since the
     # health checks can be spaced apart by interval_secs + epsilon.
-    self.forgiving_attempts = math.ceil(self.grace_period_secs / self.interval)
-
-    self.max_attempts_to_running = self.forgiving_attempts + self.min_consecutive_successes
+    self.forgiving_attempts = math.ceil(self.grace_period_secs / float(self.interval))
+
+    # In the older version (without min_consecutive_successes) it is possible for a task
+    # to make limping progress where the health checks fail all the time but never breach
+    # the max_consecutive_failures limit and end up updated successfully.
+    # Also a task can survive failures during initial_interval_secs and an additional
+    # max_consecutive_failures and still update successfully.
+
+    # Although initial_interval_secs is supposed to count for the task warm up time, to be
+    # backward compatible add max_consecutive_failures to the max_attempts_to_running.
+    self.max_attempts_to_running = (self.forgiving_attempts
+        + self.max_consecutive_failures
+        + self.min_consecutive_successes)
     self.running = False
     self.healthy, self.reason = True, None
 
@@ -164,15 +174,14 @@ class ThreadedHealthChecker(ExceptionalThread):
     if not self.running:
       attempts_remaining = self.max_attempts_to_running - self.attempts
       successes_needed = self.min_consecutive_successes - self.current_consecutive_successes
-      if successes_needed > attempts_remaining:
+      if attempts_remaining > 1 and successes_needed > attempts_remaining:
         return True
     return False
 
   def _should_ignore_failure(self):
-    if not self.running:
-      if self.attempts <= self.forgiving_attempts:
-        log.warning('Ignoring failure of attempt: %s' % self.attempts)
-        return True
+    if self.attempts <= self.forgiving_attempts:
+      log.warning('Ignoring failure of attempt: %s' % self.attempts)
+      return True
     return False
 
   def _should_enforce_deadline(self):
@@ -192,7 +201,7 @@ class ThreadedHealthChecker(ExceptionalThread):
       return self.healthy, self.reason
 
     is_healthy, reason = self._perform_check_if_not_disabled()
-    if not self.running:
+    if self.attempts <= self.max_attempts_to_running:
       self.attempts += 1
     self._maybe_update_health_check_count(is_healthy, reason)
     return is_healthy, reason

http://git-wip-us.apache.org/repos/asf/aurora/blob/d05ea2bc/src/test/python/apache/aurora/executor/common/test_health_checker.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/common/test_health_checker.py b/src/test/python/apache/aurora/executor/common/test_health_checker.py
index e2a7f16..86a71c7 100644
--- a/src/test/python/apache/aurora/executor/common/test_health_checker.py
+++ b/src/test/python/apache/aurora/executor/common/test_health_checker.py
@@ -98,6 +98,60 @@ class TestHealthChecker(unittest.TestCase):
     hct.stop()
     assert self._checker.health.call_count == 2
 
+  def test_ignore_failures_after_running_inside_grace_period(self):
+    '''Grace period is 2 x interval and health checks succeed then fail.'''
+
+    self.append_health_checks(True)
+    self.append_health_checks(False)
+    hct = HealthChecker(
+              self._checker.health,
+              interval_secs=self.interval_secs,
+              clock=self._clock)
+    hct.start()
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+    assert hct.threaded_health_checker.running is True
+    assert hct.threaded_health_checker.current_consecutive_failures == 0
+    self._clock.tick(self.interval_secs)
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+    assert hct.threaded_health_checker.running is True
+    assert hct.threaded_health_checker.current_consecutive_failures == 0
+    hct.stop()
+    assert self._checker.health.call_count == 2
+
+  def test_does_not_ignores_failures_after_running_outside_grace_period(self):
+    '''Grace period is 2 x interval and health checks succeed then fail.'''
+
+    self.append_health_checks(True)
+    self.append_health_checks(False, num_calls=2)
+    hct = HealthChecker(
+              self._checker.health,
+              interval_secs=self.interval_secs,
+              clock=self._clock)
+    hct.start()
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+    assert hct.threaded_health_checker.running is True
+    assert hct.threaded_health_checker.current_consecutive_failures == 0
+    self._clock.tick(self.interval_secs)
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+    assert hct.threaded_health_checker.running is True
+    assert hct.threaded_health_checker.current_consecutive_failures == 0
+    self._clock.tick(self.interval_secs)
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Failed health check! reason', TaskState.Value('TASK_FAILED'))
+    assert hct.threaded_health_checker.running is True
+    assert hct.threaded_health_checker.current_consecutive_failures == 1
+    hct.stop()
+    assert self._checker.health.call_count == 3
+
   def test_grace_period_2x_failure(self):
     '''
       Grace period is 2 x interval and all health checks fail.
@@ -156,6 +210,68 @@ class TestHealthChecker(unittest.TestCase):
     hct.stop()
     assert self._checker.health.call_count == 3
 
+  def test_include_max_failure_to_forgiving_attempts(self):
+    '''
+    Health checks fail but never breaches `max_consecutive_failures`
+    '''
+    max_consecutive_failures = 4
+
+    # health checks fail within grace period
+    self.append_health_checks(False, num_calls=2)
+
+    # health checks fails max_consecutive_failures times and then succeeds
+    self.append_health_checks(False, num_calls=max_consecutive_failures)
+    self.append_health_checks(True)
+
+    # health checks fails max_consecutive_failures times and then succeeds
+    self.append_health_checks(False, num_calls=max_consecutive_failures)
+    self.append_health_checks(False)
+
+    hct = HealthChecker(
+              self._checker.health,
+              interval_secs=self.interval_secs,
+              max_consecutive_failures=max_consecutive_failures,
+              clock=self._clock)
+    hct.start()
+
+    # failures ignored inside grace period
+    for _ in range(2):
+      assert self._clock.converge(threads=[hct.threaded_health_checker])
+      self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+      assert hct.status == StatusResult(None, TaskState.Value('TASK_STARTING'))
+      assert hct.threaded_health_checker.running is False
+      self._clock.tick(self.interval_secs)
+
+    # failures never breach max
+    for _ in range(max_consecutive_failures):
+      assert self._clock.converge(threads=[hct.threaded_health_checker])
+      self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+      assert hct.status == StatusResult(None, TaskState.Value('TASK_STARTING'))
+      assert hct.threaded_health_checker.running is False
+      self._clock.tick(self.interval_secs)
+
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+    assert hct.threaded_health_checker.running is True
+    self._clock.tick(self.interval_secs)
+
+    # failures breach max, causes task failure
+    for _ in range(max_consecutive_failures):
+      assert self._clock.converge(threads=[hct.threaded_health_checker])
+      self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+      assert hct.status == StatusResult('Task is healthy.', TaskState.Value('TASK_RUNNING'))
+      assert hct.threaded_health_checker.running is True
+      self._clock.tick(self.interval_secs)
+
+    assert self._clock.converge(threads=[hct.threaded_health_checker])
+    self._clock.assert_waiting(hct.threaded_health_checker, self.interval_secs)
+    assert hct.status == StatusResult('Failed health check! reason', TaskState.Value('TASK_FAILED'))
+    assert hct.threaded_health_checker.running is True
+
+    hct.stop()
+    assert self._checker.health.call_count == 12
+
   def test_initial_interval_whatev(self):
     self.append_health_checks(False, 2)
     hct = HealthChecker(
@@ -217,29 +333,39 @@ class TestHealthChecker(unittest.TestCase):
     '''Verify that health check is failed fast'''
     grace_period_secs = self.initial_interval_secs
     interval_secs = self.interval_secs
-    self.append_health_checks(False, num_calls=3)
+    self.append_health_checks(False, num_calls=5)
     hct = HealthChecker(
         self._checker.health,
         interval_secs=interval_secs,
         grace_period_secs=grace_period_secs,
         max_consecutive_failures=2,
-        min_consecutive_successes=2,
+        min_consecutive_successes=4,
         clock=self._clock)
     hct.start()
 
-    # 3 consecutive health check failures causes fail-fast
-    self._clock.converge(threads=[hct.threaded_health_checker])
-    self._clock.assert_waiting(hct.threaded_health_checker, interval_secs)
-    assert hct.status == StatusResult(None, TaskState.Value('TASK_STARTING'))
     # failure is ignored inside grace_period_secs
-    assert hct.metrics.sample()['consecutive_failures'] == 0
-    self._clock.tick(interval_secs)
+    for _ in range(2):
+      self._clock.converge(threads=[hct.threaded_health_checker])
+      self._clock.assert_waiting(hct.threaded_health_checker, interval_secs)
+      assert hct.status == StatusResult(None, TaskState.Value('TASK_STARTING'))
+      assert hct.metrics.sample()['consecutive_failures'] == 0
+      self._clock.tick(interval_secs)
+
+    # 3 consecutive health check failures causes fail-fast
+    for attempt in range(2):
+      self._clock.converge(threads=[hct.threaded_health_checker])
+      self._clock.assert_waiting(hct.threaded_health_checker, interval_secs)
+      assert hct.status == StatusResult(None, TaskState.Value('TASK_STARTING'))
+      # failure is not ignored outside grace_period_secs
+      assert hct.metrics.sample()['consecutive_failures'] == (attempt + 1)
+      self._clock.tick(interval_secs)
+
     self._clock.converge(threads=[hct.threaded_health_checker])
     self._clock.assert_waiting(hct.threaded_health_checker, interval_secs)
     assert hct.status == StatusResult('Failed health check! reason', TaskState.Value('TASK_FAILED'))
-    assert hct.metrics.sample()['consecutive_failures'] == 1
+    assert hct.metrics.sample()['consecutive_failures'] == 3
     hct.stop()
-    assert self._checker.health.call_count == 2
+    assert self._checker.health.call_count == 5
 
   @pytest.mark.skipif('True', reason='Flaky test (AURORA-1182)')
   def test_health_checker_metrics(self):
@@ -607,48 +733,65 @@ class TestThreadedHealthChecker(unittest.TestCase):
 
   def test_maybe_update_health_check_count_reset_count(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.attempts = hc.forgiving_attempts
 
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason-1')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason-3')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
 
-  def test_maybe_update_health_check_count_ignore_failures_before_callback(self):
+  def test_maybe_update_health_check_count_ignore_failures_within_grace_period(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = False
+
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
-  def test_maybe_update_health_check_count_dont_ignore_failures_after_callback(self):
+    hc.attempts += 1
+    hc._maybe_update_health_check_count(False, 'reason-3')
+    assert hc.current_consecutive_failures == 1
+    assert hc.current_consecutive_successes == 0
+
+    hc.attempts += 1
+    hc._maybe_update_health_check_count(False, 'reason-4')
+    assert hc.current_consecutive_failures == 2
+    assert hc.current_consecutive_successes == 0
+
+  def test_maybe_update_health_check_count_dont_ignore_failures_after_grace_period(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.attempts = hc.forgiving_attempts
 
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 2
     assert hc.current_consecutive_successes == 0
@@ -668,25 +811,40 @@ class TestThreadedHealthChecker(unittest.TestCase):
 
     hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
+    assert hc.current_consecutive_failures == 0
+    assert hc.current_consecutive_successes == 0
+    assert hc.running is False
+
+    hc.attempts += 1
+    hc._maybe_update_health_check_count(False, 'reason-3')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
     assert hc.running is False
+
+    hc.attempts += 1
+    hc._maybe_update_health_check_count(False, 'reason-4')
+    assert hc.current_consecutive_failures == 2
+    assert hc.current_consecutive_successes == 0
+    assert hc.running is False
     assert hc.healthy is False
-    assert hc.reason == 'reason-2'
+    assert hc.reason == 'reason-4'
 
-  def test_maybe_update_health_check_count_max_failures(self):
+  def test_maybe_update_health_check_count_max_failures_1(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.current_consecutive_successes = 1
+    hc.attempts = hc.forgiving_attempts
 
     assert hc.current_consecutive_failures == 0
-    assert hc.current_consecutive_successes == 0
+    assert hc.current_consecutive_successes == 1
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 2
     assert hc.current_consecutive_successes == 0
@@ -700,18 +858,21 @@ class TestThreadedHealthChecker(unittest.TestCase):
     assert hc.running is False
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
     assert hc.running is False
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 2
     assert hc.running is True
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 3
@@ -735,11 +896,11 @@ class TestThreadedHealthChecker(unittest.TestCase):
   def test_run_failure(self):
     self.health.return_value = (False, 'failure')
     mock_is_set = mock.Mock(spec=threading._Event.is_set)
-    liveness = [False, False, False, True]
+    liveness = [False, False, False, False, True]
     mock_is_set.side_effect = lambda: liveness.pop(0)
     self.health_checker.threaded_health_checker.dead.is_set = mock_is_set
     self.health_checker.threaded_health_checker.run()
-    assert self.clock.sleep.call_count == 3
+    assert self.clock.sleep.call_count == 4
     assert self.health_checker.threaded_health_checker.current_consecutive_failures == 2
     assert self.health_checker.threaded_health_checker.current_consecutive_successes == 0
     assert self.health_checker.threaded_health_checker.running is False
@@ -747,19 +908,19 @@ class TestThreadedHealthChecker(unittest.TestCase):
     assert self.health_checker.threaded_health_checker.reason == 'failure'
 
   def test_run_failure_unhealthy_when_failfast(self):
-    health_status = [(False, 'failure-1'), (True, None), (False, 'failure-3')]
+    health_status = [(False, 'failure-1'), (True, None), (False, 'failure-3'), (False, 'failure-4')]
     self.health.side_effect = lambda: health_status.pop(0)
     mock_is_set = mock.Mock(spec=threading._Event.is_set)
-    liveness = [False, False, False, True]
+    liveness = [False, False, False, False, True]
     mock_is_set.side_effect = lambda: liveness.pop(0)
     self.health_checker.threaded_health_checker.dead.is_set = mock_is_set
     self.health_checker.threaded_health_checker.run()
-    assert self.clock.sleep.call_count == 3
-    assert self.health_checker.threaded_health_checker.current_consecutive_failures == 1
+    assert self.clock.sleep.call_count == 4
+    assert self.health_checker.threaded_health_checker.current_consecutive_failures == 2
     assert self.health_checker.threaded_health_checker.current_consecutive_successes == 0
     assert self.health_checker.threaded_health_checker.running is False
     assert self.health_checker.threaded_health_checker.healthy is False
-    assert self.health_checker.threaded_health_checker.reason == 'failure-3'
+    assert self.health_checker.threaded_health_checker.reason == 'failure-4'
 
   def test_run_unhealthy_after_callback(self):
     health_status = [(True, None), (True, None), (False, 'failure-4'), (False, 'failure-5')]
@@ -845,48 +1006,55 @@ class TestThreadedHealthCheckerWithDefaults(unittest.TestCase):
 
   def test_maybe_update_health_check_count_reset_count(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.attempts = hc.forgiving_attempts
 
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason-1')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason-3')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
 
-  def test_maybe_update_health_check_count_ignore_failures_before_callback(self):
+  def test_maybe_update_health_check_count_ignore_failures_inside_grace_period(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = False
+
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
-  def test_maybe_update_health_check_count_dont_ignore_failures_after_callback(self):
+  def test_maybe_update_health_check_count_dont_ignore_failures_after_grace_period(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.attempts = hc.forgiving_attempts
 
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-2')
     assert hc.current_consecutive_failures == 2
     assert hc.current_consecutive_successes == 0
@@ -920,12 +1088,14 @@ class TestThreadedHealthCheckerWithDefaults(unittest.TestCase):
 
   def test_maybe_update_health_check_count_max_failures(self):
     hc = self.health_checker.threaded_health_checker
-    hc.running = True
+    hc.attempts = hc.forgiving_attempts
+    hc.current_consecutive_successes = 1
 
     assert hc.current_consecutive_failures == 0
-    assert hc.current_consecutive_successes == 0
+    assert hc.current_consecutive_successes == 1
     assert hc.healthy is True
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(False, 'reason-1')
     assert hc.current_consecutive_failures == 1
     assert hc.current_consecutive_successes == 0
@@ -939,6 +1109,7 @@ class TestThreadedHealthCheckerWithDefaults(unittest.TestCase):
     assert hc.healthy is True
     assert hc.running is False
 
+    hc.attempts += 1
     hc._maybe_update_health_check_count(True, 'reason')
     assert hc.current_consecutive_failures == 0
     assert hc.current_consecutive_successes == 1
@@ -978,6 +1149,40 @@ class TestThreadedHealthCheckerWithDefaults(unittest.TestCase):
     assert self.health_checker.threaded_health_checker.reason == 'failure'
 
   @mock.patch('apache.aurora.executor.common.health_checker.time.sleep', spec=time.sleep)
+  def test_first_success_after_grace_period_and_max_consecutive_failures(self, mock_sleep):
+    mock_sleep.return_value = None
+    health_status = [(False, 'failure-1'), (False, 'failure-2'), (True, None)]
+    self.health.side_effect = lambda: health_status.pop(0)
+    mock_is_set = mock.Mock(spec=threading._Event.is_set)
+    liveness = [False, False, False, True]
+    mock_is_set.side_effect = lambda: liveness.pop(0)
+    self.health_checker.threaded_health_checker.dead.is_set = mock_is_set
+    self.health_checker.threaded_health_checker.run()
+    assert mock_sleep.call_count == 3
+    assert self.health_checker.threaded_health_checker.current_consecutive_failures == 0
+    assert self.health_checker.threaded_health_checker.current_consecutive_successes == 1
+    assert self.health_checker.threaded_health_checker.running is True
+    assert self.health_checker.threaded_health_checker.healthy is True
+    assert self.health_checker.threaded_health_checker.reason is None
+
+  @mock.patch('apache.aurora.executor.common.health_checker.time.sleep', spec=time.sleep)
+  def test_success_then_failures_ignored_till_grace_period_ends(self, mock_sleep):
+    mock_sleep.return_value = None
+    health_status = [(True, None), (False, 'failure-2'), (False, 'failure-3')]
+    self.health.side_effect = lambda: health_status.pop(0)
+    mock_is_set = mock.Mock(spec=threading._Event.is_set)
+    liveness = [False, False, False, True]
+    mock_is_set.side_effect = lambda: liveness.pop(0)
+    self.health_checker.threaded_health_checker.dead.is_set = mock_is_set
+    self.health_checker.threaded_health_checker.run()
+    assert mock_sleep.call_count == 3
+    assert self.health_checker.threaded_health_checker.current_consecutive_failures == 1
+    assert self.health_checker.threaded_health_checker.current_consecutive_successes == 0
+    assert self.health_checker.threaded_health_checker.running is True
+    assert self.health_checker.threaded_health_checker.healthy is False
+    assert self.health_checker.threaded_health_checker.reason == 'failure-3'
+
+  @mock.patch('apache.aurora.executor.common.health_checker.time.sleep', spec=time.sleep)
   def test_run_failure_unhealthy_when_failfast(self, mock_sleep):
     mock_sleep.return_value = None
     health_status = [(False, 'failure-1'), (False, 'failure-2'), (False, 'failure-3')]