9303: Fetch arv_node before trying to shut down node, because monitor actor may
[arvados.git] / services / nodemanager / arvnodeman / computenode / dispatch / __init__.py
index 0fab1b0fec5f3e0fd0696460089028ab244cfb66..a950210aa89c0bbf18e9df64815393bfae71c65a 100644 (file)
@@ -10,75 +10,73 @@ import libcloud.common.types as cloud_types
 import pykka
 
 from .. import \
-    arvados_node_fqdn, arvados_node_mtime, arvados_timestamp, timestamp_fresh
+    arvados_node_fqdn, arvados_node_mtime, arvados_timestamp, timestamp_fresh, \
+    arvados_node_missing, RetryMixin
 from ...clientactor import _notify_subscribers
 from ... import config
+from .transitions import transitions
 
-class ComputeNodeStateChangeBase(config.actor_class):
+class ComputeNodeStateChangeBase(config.actor_class, RetryMixin):
     """Base class for actors that change a compute node's state.
 
     This base class takes care of retrying changes and notifying
     subscribers when the change is finished.
     """
-    def __init__(self, logger_name, cloud_client, timer_actor,
+    def __init__(self, cloud_client, arvados_client, timer_actor,
                  retry_wait, max_retry_wait):
         super(ComputeNodeStateChangeBase, self).__init__()
-        self._later = self.actor_ref.proxy()
-        self._logger = logging.getLogger(logger_name)
-        self._cloud = cloud_client
-        self._timer = timer_actor
-        self.min_retry_wait = retry_wait
-        self.max_retry_wait = max_retry_wait
-        self.retry_wait = retry_wait
+        RetryMixin.__init__(self, retry_wait, max_retry_wait,
+                            None, cloud_client, timer_actor)
+        self._later = self.actor_ref.tell_proxy()
+        self._arvados = arvados_client
         self.subscribers = set()
 
-    @staticmethod
-    def _retry(errors=()):
-        """Retry decorator for an actor method that makes remote requests.
-
-        Use this function to decorator an actor method, and pass in a
-        tuple of exceptions to catch.  This decorator will schedule
-        retries of that method with exponential backoff if the
-        original method raises a known cloud driver error, or any of the
-        given exception types.
-        """
-        def decorator(orig_func):
-            @functools.wraps(orig_func)
-            def retry_wrapper(self, *args, **kwargs):
-                start_time = time.time()
-                try:
-                    orig_func(self, *args, **kwargs)
-                except Exception as error:
-                    if not (isinstance(error, errors) or
-                            self._cloud.is_cloud_exception(error)):
-                        raise
-                    self._logger.warning(
-                        "Client error: %s - waiting %s seconds",
-                        error, self.retry_wait)
-                    self._timer.schedule(start_time + self.retry_wait,
-                                         getattr(self._later,
-                                                 orig_func.__name__),
-                                         *args, **kwargs)
-                    self.retry_wait = min(self.retry_wait * 2,
-                                          self.max_retry_wait)
-                else:
-                    self.retry_wait = self.min_retry_wait
-            return retry_wrapper
-        return decorator
+    def _set_logger(self):
+        self._logger = logging.getLogger("%s.%s" % (self.__class__.__name__, self.actor_urn[33:]))
+
+    def on_start(self):
+        self._set_logger()
 
     def _finished(self):
-        _notify_subscribers(self._later, self.subscribers)
+        if self.subscribers is None:
+            raise Exception("Actor tried to finish twice")
+        _notify_subscribers(self.actor_ref.proxy(), self.subscribers)
         self.subscribers = None
+        self._logger.info("finished")
 
     def subscribe(self, subscriber):
         if self.subscribers is None:
             try:
-                subscriber(self._later)
+                subscriber(self.actor_ref.proxy())
             except pykka.ActorDeadError:
                 pass
         else:
             self.subscribers.add(subscriber)
 
+    def _clean_arvados_node(self, arvados_node, explanation):
+        return self._arvados.nodes().update(
+            uuid=arvados_node['uuid'],
+            body={'hostname': None,
+                  'ip_address': None,
+                  'slot_number': None,
+                  'first_ping_at': None,
+                  'last_ping_at': None,
+                  'properties': {},
+                  'info': {'ec2_instance_id': None,
+                           'last_action': explanation}},
+            ).execute()
+
+    @staticmethod
+    def _finish_on_exception(orig_func):
+        @functools.wraps(orig_func)
+        def finish_wrapper(self, *args, **kwargs):
+            try:
+                return orig_func(self, *args, **kwargs)
+            except Exception as error:
+                self._logger.error("Actor error %s", error)
+                self._finished()
+        return finish_wrapper
+
 
 class ComputeNodeSetupActor(ComputeNodeStateChangeBase):
     """Actor to create and set up a cloud compute node.
@@ -93,9 +91,8 @@ class ComputeNodeSetupActor(ComputeNodeStateChangeBase):
                  cloud_size, arvados_node=None,
                  retry_wait=1, max_retry_wait=180):
         super(ComputeNodeSetupActor, self).__init__(
-            'arvnodeman.nodeup', cloud_client, timer_actor,
+            cloud_client, arvados_client, timer_actor,
             retry_wait, max_retry_wait)
-        self._arvados = arvados_client
         self.cloud_size = cloud_size
         self.arvados_node = None
         self.cloud_node = None
@@ -104,35 +101,57 @@ class ComputeNodeSetupActor(ComputeNodeStateChangeBase):
         else:
             self._later.prepare_arvados_node(arvados_node)
 
-    @ComputeNodeStateChangeBase._retry(config.ARVADOS_ERRORS)
+    @ComputeNodeStateChangeBase._finish_on_exception
+    @RetryMixin._retry(config.ARVADOS_ERRORS)
     def create_arvados_node(self):
         self.arvados_node = self._arvados.nodes().create(body={}).execute()
         self._later.create_cloud_node()
 
-    @ComputeNodeStateChangeBase._retry(config.ARVADOS_ERRORS)
+    @ComputeNodeStateChangeBase._finish_on_exception
+    @RetryMixin._retry(config.ARVADOS_ERRORS)
     def prepare_arvados_node(self, node):
-        self.arvados_node = self._arvados.nodes().update(
-            uuid=node['uuid'],
-            body={'hostname': None,
-                  'ip_address': None,
-                  'slot_number': None,
-                  'first_ping_at': None,
-                  'last_ping_at': None,
-                  'info': {'ec2_instance_id': None,
-                           'last_action': "Prepared by Node Manager"}}
-            ).execute()
+        self.arvados_node = self._clean_arvados_node(
+            node, "Prepared by Node Manager")
         self._later.create_cloud_node()
 
-    @ComputeNodeStateChangeBase._retry()
+    @ComputeNodeStateChangeBase._finish_on_exception
+    @RetryMixin._retry()
     def create_cloud_node(self):
-        self._logger.info("Creating cloud node with size %s.",
+        self._logger.info("Sending create_node request for node size %s.",
                           self.cloud_size.name)
         self.cloud_node = self._cloud.create_node(self.cloud_size,
                                                   self.arvados_node)
+        if not self.cloud_node.size:
+             self.cloud_node.size = self.cloud_size
         self._logger.info("Cloud node %s created.", self.cloud_node.id)
+        self._later.update_arvados_node_properties()
+
+    @ComputeNodeStateChangeBase._finish_on_exception
+    @RetryMixin._retry(config.ARVADOS_ERRORS)
+    def update_arvados_node_properties(self):
+        """Tell Arvados some details about the cloud node.
+
+        Currently we only include size/price from our request, which
+        we already knew before create_cloud_node(), but doing it here
+        gives us an opportunity to provide more detail from
+        self.cloud_node, too.
+        """
+        self.arvados_node['properties']['cloud_node'] = {
+            # Note this 'size' is the node size we asked the cloud
+            # driver to create -- not necessarily equal to the size
+            # reported by the cloud driver for the node that was
+            # created.
+            'size': self.cloud_size.id,
+            'price': self.cloud_size.price,
+        }
+        self.arvados_node = self._arvados.nodes().update(
+            uuid=self.arvados_node['uuid'],
+            body={'properties': self.arvados_node['properties']},
+        ).execute()
+        self._logger.info("%s updated properties.", self.arvados_node['uuid'])
         self._later.post_create()
 
-    @ComputeNodeStateChangeBase._retry()
+    @RetryMixin._retry()
     def post_create(self):
         self._cloud.post_create_node(self.cloud_node)
         self._logger.info("%s post-create work done.", self.cloud_node.id)
@@ -150,7 +169,11 @@ class ComputeNodeShutdownActor(ComputeNodeStateChangeBase):
 
     This actor simply destroys a cloud node, retrying as needed.
     """
-    def __init__(self, timer_actor, cloud_client, node_monitor,
+    # Reasons for a shutdown to be cancelled.
+    WINDOW_CLOSED = "shutdown window closed"
+    NODE_BROKEN = "cloud failed to shut down broken node"
+
+    def __init__(self, timer_actor, cloud_client, arvados_client, node_monitor,
                  cancellable=True, retry_wait=1, max_retry_wait=180):
         # If a ShutdownActor is cancellable, it will ask the
         # ComputeNodeMonitorActor if it's still eligible before taking each
@@ -158,47 +181,67 @@ class ComputeNodeShutdownActor(ComputeNodeStateChangeBase):
         # eligible.  Normal shutdowns based on job demand should be
         # cancellable; shutdowns based on node misbehavior should not.
         super(ComputeNodeShutdownActor, self).__init__(
-            'arvnodeman.nodedown', cloud_client, timer_actor,
+            cloud_client, arvados_client, timer_actor,
             retry_wait, max_retry_wait)
         self._monitor = node_monitor.proxy()
         self.cloud_node = self._monitor.cloud_node.get()
         self.cancellable = cancellable
+        self.cancel_reason = None
         self.success = None
 
+    def _set_logger(self):
+        self._logger = logging.getLogger("%s.%s.%s" % (self.__class__.__name__, self.actor_urn[33:], self.cloud_node.name))
+
     def on_start(self):
+        super(ComputeNodeShutdownActor, self).on_start()
         self._later.shutdown_node()
 
-    def cancel_shutdown(self):
-        self.success = False
-        self._finished()
+    def _arvados_node(self):
+        return self._monitor.arvados_node.get()
+
+    def _finished(self, success_flag=None):
+        if success_flag is not None:
+            self.success = success_flag
+        return super(ComputeNodeShutdownActor, self)._finished()
 
-    def _stop_if_window_closed(orig_func):
+    def cancel_shutdown(self, reason):
+        self.cancel_reason = reason
+        self._logger.info("Shutdown cancelled: %s.", reason)
+        self._finished(success_flag=False)
+
+    def _cancel_on_exception(orig_func):
         @functools.wraps(orig_func)
-        def stop_wrapper(self, *args, **kwargs):
-            if (self.cancellable and
-                  (not self._monitor.shutdown_eligible().get())):
-                self._logger.info(
-                    "Cloud node %s shutdown cancelled - no longer eligible.",
-                    self.cloud_node.id)
-                self._later.cancel_shutdown()
-                return None
-            else:
+        def finish_wrapper(self, *args, **kwargs):
+            try:
                 return orig_func(self, *args, **kwargs)
-        return stop_wrapper
+            except Exception as error:
+                self._logger.error("Actor error %s", error)
+                self._later.cancel_shutdown("Unhandled exception %s" % error)
+        return finish_wrapper
 
-    @_stop_if_window_closed
-    @ComputeNodeStateChangeBase._retry()
+    @_cancel_on_exception
+    @RetryMixin._retry()
     def shutdown_node(self):
-        if self._cloud.destroy_node(self.cloud_node):
-            self._logger.info("Cloud node %s shut down.", self.cloud_node.id)
-            self.success = True
-            self._finished()
+        self._logger.info("Starting shutdown")
+        arv_node = self._arvados_node()
+        if not self._cloud.destroy_node(self.cloud_node):
+            if self._cloud.broken(self.cloud_node):
+                self._later.cancel_shutdown(self.NODE_BROKEN)
+                return
+            else:
+                # Force a retry.
+                raise cloud_types.LibcloudError("destroy_node failed")
+        self._logger.info("Shutdown success")
+        if arv_node is None:
+            self._finished(success_flag=True)
         else:
-            # Force a retry.
-            raise cloud_types.LibcloudError("destroy_node failed")
+            self._later.clean_arvados_node(arv_node)
 
-    # Make the decorator available to subclasses.
-    _stop_if_window_closed = staticmethod(_stop_if_window_closed)
+    @ComputeNodeStateChangeBase._finish_on_exception
+    @RetryMixin._retry(config.ARVADOS_ERRORS)
+    def clean_arvados_node(self, arvados_node):
+        self._clean_arvados_node(arvados_node, "Shut down by Node Manager")
+        self._finished(success_flag=True)
 
 
 class ComputeNodeUpdateActor(config.actor_class):
@@ -209,12 +252,6 @@ class ComputeNodeUpdateActor(config.actor_class):
     this to perform maintenance tasks on themselves.  Having a
     dedicated actor for this gives us the opportunity to control the
     flow of requests; e.g., by backing off when errors occur.
-
-    This actor is most like a "traditional" Pykka actor: there's no
-    subscribing, but instead methods return real driver results.  If
-    you're interested in those results, you should get them from the
-    Future that the proxy method returns.  Be prepared to handle exceptions
-    from the cloud driver when you do.
     """
     def __init__(self, cloud_factory, max_retry_wait=180):
         super(ComputeNodeUpdateActor, self).__init__()
@@ -223,6 +260,12 @@ class ComputeNodeUpdateActor(config.actor_class):
         self.error_streak = 0
         self.next_request_time = time.time()
 
+    def _set_logger(self):
+        self._logger = logging.getLogger("%s.%s" % (self.__class__.__name__, self.actor_urn[33:]))
+
+    def on_start(self):
+        self._set_logger()
+
     def _throttle_errors(orig_func):
         @functools.wraps(orig_func)
         def throttle_wrapper(self, *args, **kwargs):
@@ -233,10 +276,12 @@ class ComputeNodeUpdateActor(config.actor_class):
             try:
                 result = orig_func(self, *args, **kwargs)
             except Exception as error:
-                self.error_streak += 1
-                self.next_request_time += min(2 ** self.error_streak,
-                                              self.max_retry_wait)
-                raise
+                if self._cloud.is_cloud_exception(error):
+                    self.error_streak += 1
+                    self.next_request_time += min(2 ** self.error_streak,
+                                                  self.max_retry_wait)
+                self._logger.warn(
+                    "Unhandled exception: %s", error, exc_info=error)
             else:
                 self.error_streak = 0
                 return result
@@ -255,26 +300,36 @@ class ComputeNodeMonitorActor(config.actor_class):
     for shutdown.
     """
     def __init__(self, cloud_node, cloud_node_start_time, shutdown_timer,
-                 cloud_fqdn_func, timer_actor, update_actor, arvados_node=None,
-                 poll_stale_after=600, node_stale_after=3600):
+                 cloud_fqdn_func, timer_actor, update_actor, cloud_client,
+                 arvados_node=None, poll_stale_after=600, node_stale_after=3600,
+                 boot_fail_after=1800
+    ):
         super(ComputeNodeMonitorActor, self).__init__()
-        self._later = self.actor_ref.proxy()
-        self._logger = logging.getLogger('arvnodeman.computenode')
+        self._later = self.actor_ref.tell_proxy()
         self._last_log = None
         self._shutdowns = shutdown_timer
         self._cloud_node_fqdn = cloud_fqdn_func
         self._timer = timer_actor
         self._update = update_actor
+        self._cloud = cloud_client
         self.cloud_node = cloud_node
         self.cloud_node_start_time = cloud_node_start_time
         self.poll_stale_after = poll_stale_after
         self.node_stale_after = node_stale_after
+        self.boot_fail_after = boot_fail_after
         self.subscribers = set()
         self.arvados_node = None
         self._later.update_arvados_node(arvados_node)
         self.last_shutdown_opening = None
         self._later.consider_shutdown()
 
+    def _set_logger(self):
+        self._logger = logging.getLogger("%s.%s.%s" % (self.__class__.__name__, self.actor_urn[33:], self.cloud_node.name))
+
+    def on_start(self):
+        self._set_logger()
+        self._timer.schedule(self.cloud_node_start_time + self.boot_fail_after, self._later.consider_shutdown)
+
     def subscribe(self, subscriber):
         self.subscribers.add(subscriber)
 
@@ -284,52 +339,113 @@ class ComputeNodeMonitorActor(config.actor_class):
         self._last_log = msg
         self._logger.debug(msg, *args)
 
-    def in_state(self, *states):
-        # Return a boolean to say whether or not our Arvados node record is in
-        # one of the given states.  If state information is not
-        # available--because this node has no Arvados record, the record is
-        # stale, or the record has no state information--return None.
-        if (self.arvados_node is None) or not timestamp_fresh(
-              arvados_node_mtime(self.arvados_node), self.node_stale_after):
-            return None
+    def get_state(self):
+        """Get node state, one of ['unpaired', 'busy', 'idle', 'down']."""
+
+        # If this node is not associated with an Arvados node, return 'unpaired'.
+        if self.arvados_node is None:
+            return 'unpaired'
+
         state = self.arvados_node['crunch_worker_state']
-        if not state:
-            return None
-        result = state in states
-        if state == 'idle':
-            result = result and not self.arvados_node['job_uuid']
-        return result
+
+        # If state information is not available because it is missing or the
+        # record is stale, return 'down'.
+        if not state or not timestamp_fresh(arvados_node_mtime(self.arvados_node),
+                                            self.node_stale_after):
+            state = 'down'
+
+        # There's a window between when a node pings for the first time and the
+        # value of 'slurm_state' is synchronized by crunch-dispatch.  In this
+        # window, the node will still report as 'down'.  Check that
+        # first_ping_at is truthy and consider the node 'idle' during the
+        # initial boot grace period.
+        if (state == 'down' and
+            self.arvados_node['first_ping_at'] and
+            timestamp_fresh(self.cloud_node_start_time,
+                            self.boot_fail_after) and
+            not self._cloud.broken(self.cloud_node)):
+            state = 'idle'
+
+        # "missing" means last_ping_at is stale, this should be
+        # considered "down"
+        if arvados_node_missing(self.arvados_node, self.node_stale_after):
+            state = 'down'
+
+        # Turns out using 'job_uuid' this way is a bad idea.  The node record
+        # is assigned the job_uuid before the job is locked (which removes it
+        # from the queue) which means the job will be double-counted as both in
+        # the wishlist and but also keeping a node busy.  This end result is
+        # excess nodes being booted.
+        #if state == 'idle' and self.arvados_node['job_uuid']:
+        #    state = 'busy'
+
+        return state
+
+    def in_state(self, *states):
+        return self.get_state() in states
 
     def shutdown_eligible(self):
-        if not self._shutdowns.window_open():
-            return False
-        elif self.arvados_node is None:
-            # If this is a new, unpaired node, it's eligible for
-            # shutdown--we figure there was an error during bootstrap.
-            return timestamp_fresh(self.cloud_node_start_time,
-                                   self.node_stale_after)
+        """Determine if node is candidate for shut down.
+
+        Returns a tuple of (boolean, string) where the first value is whether
+        the node is candidate for shut down, and the second value is the
+        reason for the decision.
+        """
+
+        # Collect states and then consult state transition table whether we
+        # should shut down.  Possible states are:
+        # crunch_worker_state = ['unpaired', 'busy', 'idle', 'down']
+        # window = ["open", "closed"]
+        # boot_grace = ["boot wait", "boot exceeded"]
+        # idle_grace = ["not idle", "idle wait", "idle exceeded"]
+
+        if self.arvados_node and not timestamp_fresh(arvados_node_mtime(self.arvados_node), self.node_stale_after):
+            return (False, "node state is stale")
+
+        crunch_worker_state = self.get_state()
+
+        window = "open" if self._shutdowns.window_open() else "closed"
+
+        if timestamp_fresh(self.cloud_node_start_time, self.boot_fail_after):
+            boot_grace = "boot wait"
         else:
-            return self.in_state('idle')
+            boot_grace = "boot exceeded"
+
+        # API server side not implemented yet.
+        idle_grace = 'idle exceeded'
+
+        node_state = (crunch_worker_state, window, boot_grace, idle_grace)
+        t = transitions[node_state]
+        if t is not None:
+            # yes, shutdown eligible
+            return (True, "node state is %s" % (node_state,))
+        else:
+            # no, return a reason
+            return (False, "node state is %s" % (node_state,))
 
     def consider_shutdown(self):
-        next_opening = self._shutdowns.next_opening()
-        if self.shutdown_eligible():
-            self._debug("Node %s suggesting shutdown.", self.cloud_node.id)
-            _notify_subscribers(self._later, self.subscribers)
-        elif self._shutdowns.window_open():
-            self._debug("Node %s shutdown window open but node busy.",
-                        self.cloud_node.id)
-        elif self.last_shutdown_opening != next_opening:
-            self._debug("Node %s shutdown window closed.  Next at %s.",
-                        self.cloud_node.id, time.ctime(next_opening))
-            self._timer.schedule(next_opening, self._later.consider_shutdown)
-            self.last_shutdown_opening = next_opening
+        try:
+            eligible, reason = self.shutdown_eligible()
+            next_opening = self._shutdowns.next_opening()
+            if eligible:
+                self._debug("Suggesting shutdown because %s", reason)
+                _notify_subscribers(self.actor_ref.proxy(), self.subscribers)
+            else:
+                self._debug("Not eligible for shut down because %s", reason)
+
+                if self.last_shutdown_opening != next_opening:
+                    self._debug("Shutdown window closed.  Next at %s.",
+                                time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(next_opening)))
+                    self._timer.schedule(next_opening, self._later.consider_shutdown)
+                    self.last_shutdown_opening = next_opening
+        except Exception:
+            self._logger.exception("Unexpected exception")
 
     def offer_arvados_pair(self, arvados_node):
         first_ping_s = arvados_node.get('first_ping_at')
         if (self.arvados_node is not None) or (not first_ping_s):
             return None
-        elif ((arvados_node['ip_address'] in self.cloud_node.private_ips) and
+        elif ((arvados_node['info'].get('ec2_instance_id') == self._cloud.node_id(self.cloud_node)) and
               (arvados_timestamp(first_ping_s) >= self.cloud_node_start_time)):
             self._later.update_arvados_node(arvados_node)
             return self.cloud_node.id