/**
   * Fetches all of the latest heartbeats and updates the Cluster Coordinator as appropriate, based
   * on the heartbeats received.
   *
   * <p>Visible for testing.
   */
  protected synchronized void monitorHeartbeats() {
    final Map<NodeIdentifier, NodeHeartbeat> latestHeartbeats = getLatestHeartbeats();
    if (latestHeartbeats == null || latestHeartbeats.isEmpty()) {
      logger.debug(
          "Received no new heartbeats. Will not disconnect any nodes due to lack of heartbeat");
      return;
    }

    final StopWatch procStopWatch = new StopWatch(true);
    for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
      try {
        processHeartbeat(heartbeat);
      } catch (final Exception e) {
        clusterCoordinator.reportEvent(
            null,
            Severity.ERROR,
            "Received heartbeat from "
                + heartbeat.getNodeIdentifier()
                + " but failed to process heartbeat due to "
                + e);
        logger.error(
            "Failed to process heartbeat from {} due to {}",
            heartbeat.getNodeIdentifier(),
            e.toString());
        logger.error("", e);
      }
    }

    procStopWatch.stop();
    logger.info(
        "Finished processing {} heartbeats in {}",
        latestHeartbeats.size(),
        procStopWatch.getDuration());

    // Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat
    // interval)
    final long maxMillis = heartbeatIntervalMillis * 8;
    final long threshold = System.currentTimeMillis() - maxMillis;
    for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) {
      if (heartbeat.getTimestamp() < threshold) {
        clusterCoordinator.requestNodeDisconnect(
            heartbeat.getNodeIdentifier(),
            DisconnectionCode.LACK_OF_HEARTBEAT,
            "Latest heartbeat from Node has expired");

        try {
          removeHeartbeat(heartbeat.getNodeIdentifier());
        } catch (final Exception e) {
          logger.warn(
              "Failed to remove heartbeat for {} due to {}",
              heartbeat.getNodeIdentifier(),
              e.toString());
          logger.warn("", e);
        }
      }
    }
  }
  private void processHeartbeat(final NodeHeartbeat heartbeat) {
    final NodeIdentifier nodeId = heartbeat.getNodeIdentifier();

    // Do not process heartbeat if it's blocked by firewall.
    if (clusterCoordinator.isBlockedByFirewall(nodeId.getSocketAddress())) {
      clusterCoordinator.reportEvent(
          nodeId,
          Severity.WARNING,
          "Firewall blocked received heartbeat. Issuing disconnection request.");

      // request node to disconnect
      clusterCoordinator.requestNodeDisconnect(
          nodeId, DisconnectionCode.BLOCKED_BY_FIREWALL, "Blocked by Firewall");
      removeHeartbeat(nodeId);
      return;
    }

    final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId);
    if (connectionStatus == null) {
      // Unknown node. Issue reconnect request
      clusterCoordinator.reportEvent(
          nodeId,
          Severity.INFO,
          "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster.");
      removeHeartbeat(nodeId);

      clusterCoordinator.requestNodeConnect(nodeId, null);
      return;
    }

    final NodeConnectionState connectionState = connectionStatus.getState();
    if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED
        && connectionState == NodeConnectionState.CONNECTED) {
      // Cluster Coordinator believes that node is connected, but node does not believe so.
      clusterCoordinator.reportEvent(
          nodeId,
          Severity.WARNING,
          "Received heartbeat from node that thinks it is not yet part of the cluster,"
              + "though the Cluster Coordinator thought it was (node claimed state was "
              + heartbeat.getConnectionStatus().getState()
              + "). Marking as Disconnected and requesting that Node reconnect to cluster");
      clusterCoordinator.requestNodeConnect(nodeId, null);
      return;
    }

    if (NodeConnectionState.DISCONNECTED == connectionState) {
      // ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it
      // is
      // the only node. We allow it if it is the only node because if we have a one-node cluster,
      // then
      // we cannot manually reconnect it.
      final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode();

      // Determine whether or not the node should be allowed to be in the cluster still, depending
      // on its reason for disconnection.
      if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT
          || disconnectionCode == DisconnectionCode.UNABLE_TO_COMMUNICATE) {
        clusterCoordinator.reportEvent(
            nodeId,
            Severity.INFO,
            "Received heartbeat from node previously "
                + "disconnected due to "
                + disconnectionCode
                + ". Issuing reconnection request.");

        clusterCoordinator.requestNodeConnect(nodeId, null);
      } else {
        // disconnected nodes should not heartbeat, so we need to issue a disconnection request.
        logger.info(
            "Ignoring received heartbeat from disconnected node "
                + nodeId
                + ".  Issuing disconnection request.");
        clusterCoordinator.requestNodeDisconnect(
            nodeId,
            DisconnectionCode.HEARTBEAT_RECEIVED_FROM_DISCONNECTED_NODE,
            DisconnectionCode.HEARTBEAT_RECEIVED_FROM_DISCONNECTED_NODE.toString());
        removeHeartbeat(nodeId);
      }

      return;
    }

    if (NodeConnectionState.DISCONNECTING == connectionStatus.getState()) {
      // ignore spurious heartbeat
      removeHeartbeat(nodeId);
      return;
    }

    // first heartbeat causes status change from connecting to connected
    if (NodeConnectionState.CONNECTING == connectionState) {
      final Long connectionRequestTime = connectionStatus.getConnectionRequestTime();
      if (connectionRequestTime != null && heartbeat.getTimestamp() < connectionRequestTime) {
        clusterCoordinator.reportEvent(
            nodeId,
            Severity.INFO,
            "Received heartbeat but ignoring because it was reported before the node was last asked to reconnect.");
        removeHeartbeat(nodeId);
        return;
      }

      // connection complete
      clusterCoordinator.finishNodeConnection(nodeId);
      clusterCoordinator.reportEvent(
          nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected.");
    }

    clusterCoordinator.updateNodeRoles(nodeId, heartbeat.getRoles());
  }